You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by jp...@apache.org on 2023/02/06 16:08:20 UTC

[lucene] branch main updated: Improve document API for stored fields. (#12116)

This is an automated email from the ASF dual-hosted git repository.

jpountz pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/lucene.git


The following commit(s) were added to refs/heads/main by this push:
     new 96136b42821 Improve document API for stored fields. (#12116)
96136b42821 is described below

commit 96136b42821205e942f21a26c4992530c6909aca
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Mon Feb 6 17:08:13 2023 +0100

    Improve document API for stored fields. (#12116)
    
    Currently stored fields have to look at binaryValue(), stringValue() and
    numericValue() to guess the type of the value and then store it. This has a few
    issues:
     - If there is a problem, e.g. all of these 3 methods return null, it's
       currently discovered late, when we already passed the responsibility of
       writing data from IndexingChain to the codec.
     - numericValue() is used both for numeric doc values and storage. This makes
       it impossible to implement a `double` field that is stored and doc-valued,
       as numericValue() needs to return simultaneously a number that consists of
       the double for storage, and the long bits of the double for doc values.
     - binaryValue() is used both for sorted(_set) doc values and storage. This
       makes it impossible to implement `keyword` fields that is stored and
       doc-valued, as the field returns a non-null value for both binaryValue() and
       stringValue() and stored fields no longer know which field to store.
    
    This commit introduces `IndexableField#storedValue()`, which is used only for
    stored fields. This addresses the above issues. IndexingChain passes the
    storedValue() directly to the codec, so it's impossible for a stored fields
    format to mistakenly use binaryValue()/stringValue()/numericValue() instead of
    storedValue().
---
 lucene/CHANGES.txt                                 |   5 +-
 .../Lucene50CompressingStoredFieldsWriter.java     |  96 +++++-----
 .../simpletext/SimpleTextStoredFieldsWriter.java   | 140 ++++++++-------
 .../apache/lucene/codecs/StoredFieldsWriter.java   | 118 ++++---------
 .../Lucene90CompressingStoredFieldsWriter.java     |  96 +++++-----
 .../org/apache/lucene/document/DoubleField.java    |  43 ++++-
 .../src/java/org/apache/lucene/document/Field.java |  23 +++
 .../org/apache/lucene/document/FloatField.java     |  43 ++++-
 .../java/org/apache/lucene/document/IntField.java  |  51 +++++-
 .../java/org/apache/lucene/document/LongField.java |  49 +++++-
 .../org/apache/lucene/document/StoredValue.java    | 190 ++++++++++++++++++++
 .../org/apache/lucene/document/StringField.java    |  33 ++++
 .../java/org/apache/lucene/document/TextField.java |  22 +++
 .../org/apache/lucene/index/IndexableField.java    |   7 +
 .../org/apache/lucene/index/IndexingChain.java     |  12 +-
 .../lucene/index/SortingStoredFieldsConsumer.java  |  81 +--------
 .../apache/lucene/index/StoredFieldsConsumer.java  |  26 ++-
 .../TestCompressingStoredFieldsFormat.java         |  49 ------
 .../test/org/apache/lucene/document/TestField.java | 193 ++++++++++++++-------
 .../document/TestLongDistanceFeatureQuery.java     |  19 +-
 .../org/apache/lucene/index/TestFieldReuse.java    |   6 +
 .../apache/lucene/index/TestIndexableField.java    |  17 ++
 .../lucene/search/TestIndexOrDocValuesQuery.java   |   6 +-
 .../apache/lucene/search/TestSortOptimization.java |   4 +-
 .../lucene/search/TestSortedNumericSortField.java  |  41 ++---
 .../java/org/apache/lucene/demo/IndexFiles.java    |   2 +-
 .../lucene/index/memory/TestMemoryIndex.java       |   3 +-
 .../apache/lucene/misc/document/LazyDocument.java  |   6 +
 .../asserting/AssertingStoredFieldsFormat.java     |  36 +++-
 .../codecs/cranky/CrankyStoredFieldsFormat.java    |  46 ++++-
 .../tests/index/BaseIndexFileFormatTestCase.java   |  25 ++-
 31 files changed, 977 insertions(+), 511 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 561c860a907..2ca08528a2a 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -104,7 +104,10 @@ Other
 
 API Changes
 ---------------------
-(No changes)
+
+* GITHUB#12116: Introduce IndexableField#storedValue() to expose the value that
+  should be stored to IndexingChain without needing to guess the field's type.
+  (Adrien Grand, Robert Muir)
 
 New Features
 ---------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/compressing/Lucene50CompressingStoredFieldsWriter.java b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/compressing/Lucene50CompressingStoredFieldsWriter.java
index f3d85d99489..53104fd1c5c 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/compressing/Lucene50CompressingStoredFieldsWriter.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/compressing/Lucene50CompressingStoredFieldsWriter.java
@@ -37,7 +37,6 @@ import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.StoredFieldsWriter;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.store.ByteBuffersDataOutput;
 import org.apache.lucene.store.DataOutput;
@@ -278,67 +277,52 @@ public final class Lucene50CompressingStoredFieldsWriter extends StoredFieldsWri
   }
 
   @Override
-  public void writeField(FieldInfo info, IndexableField field) throws IOException {
+  public void writeField(FieldInfo info, int value) throws IOException {
+    ++numStoredFieldsInDoc;
+    final long infoAndBits = (((long) info.number) << TYPE_BITS) | NUMERIC_INT;
+    bufferedDocs.writeVLong(infoAndBits);
+    bufferedDocs.writeZInt(value);
+  }
 
+  @Override
+  public void writeField(FieldInfo info, long value) throws IOException {
     ++numStoredFieldsInDoc;
+    final long infoAndBits = (((long) info.number) << TYPE_BITS) | NUMERIC_LONG;
+    bufferedDocs.writeVLong(infoAndBits);
+    writeTLong(bufferedDocs, value);
+  }
 
-    int bits = 0;
-    final BytesRef bytes;
-    final String string;
-
-    Number number = field.numericValue();
-    if (number != null) {
-      if (number instanceof Byte || number instanceof Short || number instanceof Integer) {
-        bits = NUMERIC_INT;
-      } else if (number instanceof Long) {
-        bits = NUMERIC_LONG;
-      } else if (number instanceof Float) {
-        bits = NUMERIC_FLOAT;
-      } else if (number instanceof Double) {
-        bits = NUMERIC_DOUBLE;
-      } else {
-        throw new IllegalArgumentException("cannot store numeric type " + number.getClass());
-      }
-      string = null;
-      bytes = null;
-    } else {
-      bytes = field.binaryValue();
-      if (bytes != null) {
-        bits = BYTE_ARR;
-        string = null;
-      } else {
-        bits = STRING;
-        string = field.stringValue();
-        if (string == null) {
-          throw new IllegalArgumentException(
-              "field "
-                  + field.name()
-                  + " is stored but does not have binaryValue, stringValue nor numericValue");
-        }
-      }
-    }
+  @Override
+  public void writeField(FieldInfo info, float value) throws IOException {
+    ++numStoredFieldsInDoc;
+    final long infoAndBits = (((long) info.number) << TYPE_BITS) | NUMERIC_FLOAT;
+    bufferedDocs.writeVLong(infoAndBits);
+    writeZFloat(bufferedDocs, value);
+  }
 
-    final long infoAndBits = (((long) info.number) << TYPE_BITS) | bits;
+  @Override
+  public void writeField(FieldInfo info, double value) throws IOException {
+    ++numStoredFieldsInDoc;
+    final long infoAndBits = (((long) info.number) << TYPE_BITS) | NUMERIC_DOUBLE;
     bufferedDocs.writeVLong(infoAndBits);
+    writeZDouble(bufferedDocs, value);
+  }
 
-    if (bytes != null) {
-      bufferedDocs.writeVInt(bytes.length);
-      bufferedDocs.writeBytes(bytes.bytes, bytes.offset, bytes.length);
-    } else if (string != null) {
-      bufferedDocs.writeString(string);
-    } else {
-      if (number instanceof Byte || number instanceof Short || number instanceof Integer) {
-        bufferedDocs.writeZInt(number.intValue());
-      } else if (number instanceof Long) {
-        writeTLong(EndiannessReverserUtil.wrapDataOutput(bufferedDocs), number.longValue());
-      } else if (number instanceof Float) {
-        writeZFloat(EndiannessReverserUtil.wrapDataOutput(bufferedDocs), number.floatValue());
-      } else if (number instanceof Double) {
-        writeZDouble(EndiannessReverserUtil.wrapDataOutput(bufferedDocs), number.doubleValue());
-      } else {
-        throw new AssertionError("Cannot get here");
-      }
-    }
+  @Override
+  public void writeField(FieldInfo info, BytesRef value) throws IOException {
+    ++numStoredFieldsInDoc;
+    final long infoAndBits = (((long) info.number) << TYPE_BITS) | BYTE_ARR;
+    bufferedDocs.writeVLong(infoAndBits);
+    bufferedDocs.writeVInt(value.length);
+    bufferedDocs.writeBytes(value.bytes, value.offset, value.length);
+  }
+
+  @Override
+  public void writeField(FieldInfo info, String value) throws IOException {
+    ++numStoredFieldsInDoc;
+    final long infoAndBits = (((long) info.number) << TYPE_BITS) | STRING;
+    bufferedDocs.writeVLong(infoAndBits);
+    bufferedDocs.writeString(value);
   }
 
   // -0 isn't compressed.
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java
index 4fb12365050..34a270672cb 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java
@@ -20,7 +20,6 @@ import java.io.IOException;
 import org.apache.lucene.codecs.StoredFieldsWriter;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
@@ -82,72 +81,91 @@ public class SimpleTextStoredFieldsWriter extends StoredFieldsWriter {
   }
 
   @Override
-  public void writeField(FieldInfo info, IndexableField field) throws IOException {
+  public void writeField(FieldInfo info, int value) throws IOException {
+    writeField(info);
+
+    write(TYPE);
+    write(TYPE_INT);
+    newLine();
+
+    write(VALUE);
+    write(Integer.toString(value));
+    newLine();
+  }
+
+  @Override
+  public void writeField(FieldInfo info, long value) throws IOException {
+    writeField(info);
+
+    write(TYPE);
+    write(TYPE_LONG);
+    newLine();
+
+    write(VALUE);
+    write(Long.toString(value));
+    newLine();
+  }
+
+  @Override
+  public void writeField(FieldInfo info, float value) throws IOException {
+    writeField(info);
+
+    write(TYPE);
+    write(TYPE_FLOAT);
+    newLine();
+
+    write(VALUE);
+    write(Float.toString(value));
+    newLine();
+  }
+
+  @Override
+  public void writeField(FieldInfo info, double value) throws IOException {
+    writeField(info);
+
+    write(TYPE);
+    write(TYPE_DOUBLE);
+    newLine();
+
+    write(VALUE);
+    write(Double.toString(value));
+    newLine();
+  }
+
+  @Override
+  public void writeField(FieldInfo info, BytesRef value) throws IOException {
+    writeField(info);
+
+    write(TYPE);
+    write(TYPE_BINARY);
+    newLine();
+
+    write(VALUE);
+    write(value);
+    newLine();
+  }
+
+  @Override
+  public void writeField(FieldInfo info, String value) throws IOException {
+    writeField(info);
+
+    write(TYPE);
+    write(TYPE_STRING);
+    newLine();
+
+    write(VALUE);
+    write(value);
+    newLine();
+  }
+
+  private void writeField(FieldInfo info) throws IOException {
     write(FIELD);
     write(Integer.toString(info.number));
     newLine();
 
     write(NAME);
-    write(field.name());
+    write(info.name);
     newLine();
-
-    write(TYPE);
-    final Number n = field.numericValue();
-
-    if (n != null) {
-      if (n instanceof Byte || n instanceof Short || n instanceof Integer) {
-        write(TYPE_INT);
-        newLine();
-
-        write(VALUE);
-        write(Integer.toString(n.intValue()));
-        newLine();
-      } else if (n instanceof Long) {
-        write(TYPE_LONG);
-        newLine();
-
-        write(VALUE);
-        write(Long.toString(n.longValue()));
-        newLine();
-      } else if (n instanceof Float) {
-        write(TYPE_FLOAT);
-        newLine();
-
-        write(VALUE);
-        write(Float.toString(n.floatValue()));
-        newLine();
-      } else if (n instanceof Double) {
-        write(TYPE_DOUBLE);
-        newLine();
-
-        write(VALUE);
-        write(Double.toString(n.doubleValue()));
-        newLine();
-      } else {
-        throw new IllegalArgumentException("cannot store numeric type " + n.getClass());
-      }
-    } else {
-      BytesRef bytes = field.binaryValue();
-      if (bytes != null) {
-        write(TYPE_BINARY);
-        newLine();
-
-        write(VALUE);
-        write(bytes);
-        newLine();
-      } else if (field.stringValue() == null) {
-        throw new IllegalArgumentException(
-            "field "
-                + field.name()
-                + " is stored but does not have binaryValue, stringValue nor numericValue");
-      } else {
-        write(TYPE_STRING);
-        newLine();
-        write(VALUE);
-        write(field.stringValue());
-        newLine();
-      }
-    }
   }
 
   @Override
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java
index 1b527e59342..8dc68b836f7 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java
@@ -20,18 +20,13 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
 
 import java.io.Closeable;
 import java.io.IOException;
-import java.io.Reader;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Objects;
-import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.document.StoredField;
+import org.apache.lucene.document.StoredValue;
 import org.apache.lucene.index.DocIDMerger;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.index.IndexableFieldType;
 import org.apache.lucene.index.MergeState;
 import org.apache.lucene.index.StoredFieldVisitor;
 import org.apache.lucene.util.Accountable;
@@ -43,7 +38,7 @@ import org.apache.lucene.util.BytesRef;
  * <ol>
  *   <li>For every document, {@link #startDocument()} is called, informing the Codec that a new
  *       document has started.
- *   <li>{@link #writeField(FieldInfo, IndexableField)} is called for each field in the document.
+ *   <li>{@link #writeField} is called for each field in the document.
  *   <li>After all documents have been written, {@link #finish(int)} is called for
  *       verification/sanity-checks.
  *   <li>Finally the writer is closed ({@link #close()})
@@ -57,17 +52,31 @@ public abstract class StoredFieldsWriter implements Closeable, Accountable {
   protected StoredFieldsWriter() {}
 
   /**
-   * Called before writing the stored fields of the document. {@link #writeField(FieldInfo,
-   * IndexableField)} will be called for each stored field. Note that this is called even if the
-   * document has no stored fields.
+   * Called before writing the stored fields of the document. {@link #writeField} will be called for
+   * each stored field. Note that this is called even if the document has no stored fields.
    */
   public abstract void startDocument() throws IOException;
 
   /** Called when a document and all its fields have been added. */
   public void finishDocument() throws IOException {}
 
-  /** Writes a single stored field. */
-  public abstract void writeField(FieldInfo info, IndexableField field) throws IOException;
+  /** Writes a stored int value. */
+  public abstract void writeField(FieldInfo info, int value) throws IOException;
+
+  /** Writes a stored long value. */
+  public abstract void writeField(FieldInfo info, long value) throws IOException;
+
+  /** Writes a stored float value. */
+  public abstract void writeField(FieldInfo info, float value) throws IOException;
+
+  /** Writes a stored double value. */
+  public abstract void writeField(FieldInfo info, double value) throws IOException;
+
+  /** Writes a stored binary value. */
+  public abstract void writeField(FieldInfo info, BytesRef value) throws IOException;
+
+  /** Writes a stored String value. */
+  public abstract void writeField(FieldInfo info, String value) throws IOException;
 
   /**
    * Called before {@link #close()}, passing in the number of documents that were written. Note that
@@ -104,9 +113,9 @@ public abstract class StoredFieldsWriter implements Closeable, Accountable {
   /**
    * Merges in the stored fields from the readers in <code>mergeState</code>. The default
    * implementation skips over deleted documents, and uses {@link #startDocument()}, {@link
-   * #writeField(FieldInfo, IndexableField)}, and {@link #finish(int)}, returning the number of
-   * documents that were written. Implementations can override this method for more sophisticated
-   * merging (bulk-byte copying, etc).
+   * #writeField}, and {@link #finish(int)}, returning the number of documents that were written.
+   * Implementations can override this method for more sophisticated merging (bulk-byte copying,
+   * etc).
    */
   public int merge(MergeState mergeState) throws IOException {
     List<StoredFieldsMergeSub> subs = new ArrayList<>();
@@ -154,10 +163,8 @@ public abstract class StoredFieldsWriter implements Closeable, Accountable {
    * }
    * </pre>
    */
-  protected class MergeVisitor extends StoredFieldVisitor implements IndexableField {
-    BytesRef binaryValue;
-    String stringValue;
-    Number numericValue;
+  protected class MergeVisitor extends StoredFieldVisitor {
+    StoredValue storedValue;
     FieldInfo currentField;
     FieldInfos remapper;
 
@@ -177,45 +184,34 @@ public abstract class StoredFieldsWriter implements Closeable, Accountable {
 
     @Override
     public void binaryField(FieldInfo fieldInfo, byte[] value) throws IOException {
-      reset(fieldInfo);
       // TODO: can we avoid new BR here?
-      binaryValue = new BytesRef(value);
-      write();
+      writeField(remap(fieldInfo), new BytesRef(value));
     }
 
     @Override
     public void stringField(FieldInfo fieldInfo, String value) throws IOException {
-      reset(fieldInfo);
-      stringValue = Objects.requireNonNull(value, "String value should not be null");
-      write();
+      writeField(
+          remap(fieldInfo), Objects.requireNonNull(value, "String value should not be null"));
     }
 
     @Override
     public void intField(FieldInfo fieldInfo, int value) throws IOException {
-      reset(fieldInfo);
-      numericValue = value;
-      write();
+      writeField(remap(fieldInfo), value);
     }
 
     @Override
     public void longField(FieldInfo fieldInfo, long value) throws IOException {
-      reset(fieldInfo);
-      numericValue = value;
-      write();
+      writeField(remap(fieldInfo), value);
     }
 
     @Override
     public void floatField(FieldInfo fieldInfo, float value) throws IOException {
-      reset(fieldInfo);
-      numericValue = value;
-      write();
+      writeField(remap(fieldInfo), value);
     }
 
     @Override
     public void doubleField(FieldInfo fieldInfo, double value) throws IOException {
-      reset(fieldInfo);
-      numericValue = value;
-      write();
+      writeField(remap(fieldInfo), value);
     }
 
     @Override
@@ -223,55 +219,13 @@ public abstract class StoredFieldsWriter implements Closeable, Accountable {
       return Status.YES;
     }
 
-    @Override
-    public String name() {
-      return currentField.name;
-    }
-
-    @Override
-    public IndexableFieldType fieldType() {
-      return StoredField.TYPE;
-    }
-
-    @Override
-    public BytesRef binaryValue() {
-      return binaryValue;
-    }
-
-    @Override
-    public String stringValue() {
-      return stringValue;
-    }
-
-    @Override
-    public Number numericValue() {
-      return numericValue;
-    }
-
-    @Override
-    public Reader readerValue() {
-      return null;
-    }
-
-    @Override
-    public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) {
-      return null;
-    }
-
-    void reset(FieldInfo field) {
+    private FieldInfo remap(FieldInfo field) {
       if (remapper != null) {
         // field numbers are not aligned, we need to remap to the new field number
-        currentField = remapper.fieldInfo(field.name);
+        return remapper.fieldInfo(field.name);
       } else {
-        currentField = field;
+        return field;
       }
-      binaryValue = null;
-      stringValue = null;
-      numericValue = null;
-    }
-
-    void write() throws IOException {
-      writeField(currentField, this);
     }
   }
 
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/compressing/Lucene90CompressingStoredFieldsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/compressing/Lucene90CompressingStoredFieldsWriter.java
index 63afcef96d9..c797c6326f2 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene90/compressing/Lucene90CompressingStoredFieldsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene90/compressing/Lucene90CompressingStoredFieldsWriter.java
@@ -32,7 +32,6 @@ import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.DocIDMerger;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
-import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.MergeState;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.store.ByteBuffersDataInput;
@@ -267,67 +266,52 @@ public final class Lucene90CompressingStoredFieldsWriter extends StoredFieldsWri
   }
 
   @Override
-  public void writeField(FieldInfo info, IndexableField field) throws IOException {
+  public void writeField(FieldInfo info, int value) throws IOException {
+    ++numStoredFieldsInDoc;
+    final long infoAndBits = (((long) info.number) << TYPE_BITS) | NUMERIC_INT;
+    bufferedDocs.writeVLong(infoAndBits);
+    bufferedDocs.writeZInt(value);
+  }
 
+  @Override
+  public void writeField(FieldInfo info, long value) throws IOException {
     ++numStoredFieldsInDoc;
+    final long infoAndBits = (((long) info.number) << TYPE_BITS) | NUMERIC_LONG;
+    bufferedDocs.writeVLong(infoAndBits);
+    writeTLong(bufferedDocs, value);
+  }
 
-    int bits = 0;
-    final BytesRef bytes;
-    final String string;
-
-    Number number = field.numericValue();
-    if (number != null) {
-      if (number instanceof Byte || number instanceof Short || number instanceof Integer) {
-        bits = NUMERIC_INT;
-      } else if (number instanceof Long) {
-        bits = NUMERIC_LONG;
-      } else if (number instanceof Float) {
-        bits = NUMERIC_FLOAT;
-      } else if (number instanceof Double) {
-        bits = NUMERIC_DOUBLE;
-      } else {
-        throw new IllegalArgumentException("cannot store numeric type " + number.getClass());
-      }
-      string = null;
-      bytes = null;
-    } else {
-      bytes = field.binaryValue();
-      if (bytes != null) {
-        bits = BYTE_ARR;
-        string = null;
-      } else {
-        bits = STRING;
-        string = field.stringValue();
-        if (string == null) {
-          throw new IllegalArgumentException(
-              "field "
-                  + field.name()
-                  + " is stored but does not have binaryValue, stringValue nor numericValue");
-        }
-      }
-    }
+  @Override
+  public void writeField(FieldInfo info, float value) throws IOException {
+    ++numStoredFieldsInDoc;
+    final long infoAndBits = (((long) info.number) << TYPE_BITS) | NUMERIC_FLOAT;
+    bufferedDocs.writeVLong(infoAndBits);
+    writeZFloat(bufferedDocs, value);
+  }
 
-    final long infoAndBits = (((long) info.number) << TYPE_BITS) | bits;
+  @Override
+  public void writeField(FieldInfo info, double value) throws IOException {
+    ++numStoredFieldsInDoc;
+    final long infoAndBits = (((long) info.number) << TYPE_BITS) | NUMERIC_DOUBLE;
     bufferedDocs.writeVLong(infoAndBits);
+    writeZDouble(bufferedDocs, value);
+  }
 
-    if (bytes != null) {
-      bufferedDocs.writeVInt(bytes.length);
-      bufferedDocs.writeBytes(bytes.bytes, bytes.offset, bytes.length);
-    } else if (string != null) {
-      bufferedDocs.writeString(string);
-    } else {
-      if (number instanceof Byte || number instanceof Short || number instanceof Integer) {
-        bufferedDocs.writeZInt(number.intValue());
-      } else if (number instanceof Long) {
-        writeTLong(bufferedDocs, number.longValue());
-      } else if (number instanceof Float) {
-        writeZFloat(bufferedDocs, number.floatValue());
-      } else if (number instanceof Double) {
-        writeZDouble(bufferedDocs, number.doubleValue());
-      } else {
-        throw new AssertionError("Cannot get here");
-      }
-    }
+  @Override
+  public void writeField(FieldInfo info, BytesRef value) throws IOException {
+    ++numStoredFieldsInDoc;
+    final long infoAndBits = (((long) info.number) << TYPE_BITS) | BYTE_ARR;
+    bufferedDocs.writeVLong(infoAndBits);
+    bufferedDocs.writeVInt(value.length);
+    bufferedDocs.writeBytes(value.bytes, value.offset, value.length);
+  }
+
+  @Override
+  public void writeField(FieldInfo info, String value) throws IOException {
+    ++numStoredFieldsInDoc;
+    final long infoAndBits = (((long) info.number) << TYPE_BITS) | STRING;
+    bufferedDocs.writeVLong(infoAndBits);
+    bufferedDocs.writeString(value);
   }
 
   // -0 isn't compressed.
diff --git a/lucene/core/src/java/org/apache/lucene/document/DoubleField.java b/lucene/core/src/java/org/apache/lucene/document/DoubleField.java
index dc4b4ce6e72..911d7c42a7f 100644
--- a/lucene/core/src/java/org/apache/lucene/document/DoubleField.java
+++ b/lucene/core/src/java/org/apache/lucene/document/DoubleField.java
@@ -29,9 +29,8 @@ import org.apache.lucene.util.NumericUtils;
 
 /**
  * Field that stores a per-document <code>double</code> value for scoring, sorting or value
- * retrieval and index the field for fast range filters. If you also need to store the value, you
- * should add a separate {@link StoredField} instance. If you need more fine-grained control you can
- * use {@link DoublePoint} and {@link DoubleDocValuesField}.
+ * retrieval and index the field for fast range filters. If you need more fine-grained control you
+ * can use {@link DoublePoint}, {@link DoubleDocValuesField} and {@link StoredField}.
  *
  * <p>This field defines static factory methods for creating common queries:
  *
@@ -46,23 +45,51 @@ import org.apache.lucene.util.NumericUtils;
 public final class DoubleField extends Field {
 
   private static final FieldType FIELD_TYPE = new FieldType();
+  private static final FieldType FIELD_TYPE_STORED;
 
   static {
     FIELD_TYPE.setDimensions(1, Double.BYTES);
     FIELD_TYPE.setDocValuesType(DocValuesType.SORTED_NUMERIC);
     FIELD_TYPE.freeze();
+
+    FIELD_TYPE_STORED = new FieldType(FIELD_TYPE);
+    FIELD_TYPE_STORED.setStored(true);
+    FIELD_TYPE_STORED.freeze();
   }
 
+  private final StoredValue storedValue;
+
   /**
    * Creates a new DoubleField, indexing the provided point and storing it as a DocValue
    *
    * @param name field name
    * @param value the double value
    * @throws IllegalArgumentException if the field name or value is null.
+   * @deprecated Use {@link #DoubleField(String, int, Field.Store)} with {@link Field.Store#NO}
+   *     instead.
    */
+  @Deprecated
   public DoubleField(String name, double value) {
-    super(name, FIELD_TYPE);
+    this(name, value, Field.Store.NO);
+  }
+
+  /**
+   * Creates a new DoubleField, indexing the provided point, storing it as a DocValue, and
+   * optionally storing it as a stored field.
+   *
+   * @param name field name
+   * @param value the double value
+   * @param stored whether to store the field
+   * @throws IllegalArgumentException if the field name or value is null.
+   */
+  public DoubleField(String name, double value, Field.Store stored) {
+    super(name, stored == Field.Store.YES ? FIELD_TYPE_STORED : FIELD_TYPE);
     fieldsData = NumericUtils.doubleToSortableLong(value);
+    if (stored == Field.Store.YES) {
+      storedValue = new StoredValue(value);
+    } else {
+      storedValue = null;
+    }
   }
 
   @Override
@@ -77,6 +104,11 @@ public final class DoubleField extends Field {
     return NumericUtils.sortableLongToDouble(numericValue().longValue());
   }
 
+  @Override
+  public StoredValue storedValue() {
+    return storedValue;
+  }
+
   @Override
   public String toString() {
     return getClass().getSimpleName() + " <" + name + ':' + getValueAsDouble() + '>';
@@ -85,6 +117,9 @@ public final class DoubleField extends Field {
   @Override
   public void setDoubleValue(double value) {
     super.setLongValue(NumericUtils.doubleToSortableLong(value));
+    if (storedValue != null) {
+      storedValue.setDoubleValue(value);
+    }
   }
 
   @Override
diff --git a/lucene/core/src/java/org/apache/lucene/document/Field.java b/lucene/core/src/java/org/apache/lucene/document/Field.java
index 94e86c7bcdb..4a390bcbf65 100644
--- a/lucene/core/src/java/org/apache/lucene/document/Field.java
+++ b/lucene/core/src/java/org/apache/lucene/document/Field.java
@@ -597,4 +597,27 @@ public class Field implements IndexableField {
     /** Do not store the field value in the index. */
     NO
   }
+
+  @Override
+  public StoredValue storedValue() {
+    if (fieldType().stored() == false) {
+      return null;
+    } else if (fieldsData == null) {
+      throw new IllegalArgumentException("fieldsData is unset");
+    } else if (fieldsData instanceof Integer) {
+      return new StoredValue((int) fieldsData);
+    } else if (fieldsData instanceof Long) {
+      return new StoredValue((long) fieldsData);
+    } else if (fieldsData instanceof Float) {
+      return new StoredValue((float) fieldsData);
+    } else if (fieldsData instanceof Double) {
+      return new StoredValue((double) fieldsData);
+    } else if (fieldsData instanceof BytesRef) {
+      return new StoredValue((BytesRef) fieldsData);
+    } else if (fieldsData instanceof String) {
+      return new StoredValue((String) fieldsData);
+    } else {
+      throw new IllegalStateException("Cannot store value of type " + fieldsData.getClass());
+    }
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/document/FloatField.java b/lucene/core/src/java/org/apache/lucene/document/FloatField.java
index 437cf57d1ce..eb1d7d400af 100644
--- a/lucene/core/src/java/org/apache/lucene/document/FloatField.java
+++ b/lucene/core/src/java/org/apache/lucene/document/FloatField.java
@@ -29,9 +29,8 @@ import org.apache.lucene.util.NumericUtils;
 
 /**
  * Field that stores a per-document <code>float</code> value for scoring, sorting or value retrieval
- * and index the field for fast range filters. If you also need to store the value, you should add a
- * separate {@link StoredField} instance. If you need more fine-grained control you can use {@link
- * FloatPoint} and {@link FloatDocValuesField}.
+ * and index the field for fast range filters. If you need more fine-grained control you can use
+ * {@link FloatPoint}, {@link FloatDocValuesField} and {@link StoredField}.
  *
  * <p>This field defines static factory methods for creating common queries:
  *
@@ -46,23 +45,51 @@ import org.apache.lucene.util.NumericUtils;
 public final class FloatField extends Field {
 
   private static final FieldType FIELD_TYPE = new FieldType();
+  private static final FieldType FIELD_TYPE_STORED;
 
   static {
     FIELD_TYPE.setDimensions(1, Float.BYTES);
     FIELD_TYPE.setDocValuesType(DocValuesType.SORTED_NUMERIC);
     FIELD_TYPE.freeze();
+
+    FIELD_TYPE_STORED = new FieldType(FIELD_TYPE);
+    FIELD_TYPE_STORED.setStored(true);
+    FIELD_TYPE_STORED.freeze();
   }
 
+  private final StoredValue storedValue;
+
   /**
    * Creates a new FloatField, indexing the provided point and storing it as a DocValue
    *
    * @param name field name
    * @param value the float value
    * @throws IllegalArgumentException if the field name or value is null.
+   * @deprecated Use {@link #FloatField(String, int, Field.Store)} with {@link Field.Store#NO}
+   *     instead.
    */
+  @Deprecated
   public FloatField(String name, float value) {
-    super(name, FIELD_TYPE);
+    this(name, value, Field.Store.NO);
+  }
+
+  /**
+   * Creates a new FloatField, indexing the provided point, storing it as a DocValue, and optionally
+   * storing it as a stored field.
+   *
+   * @param name field name
+   * @param value the float value
+   * @param stored whether to store the field
+   * @throws IllegalArgumentException if the field name or value is null.
+   */
+  public FloatField(String name, float value, Field.Store stored) {
+    super(name, stored == Field.Store.YES ? FIELD_TYPE_STORED : FIELD_TYPE);
     fieldsData = (long) NumericUtils.floatToSortableInt(value);
+    if (stored == Field.Store.YES) {
+      storedValue = new StoredValue(value);
+    } else {
+      storedValue = null;
+    }
   }
 
   @Override
@@ -77,6 +104,11 @@ public final class FloatField extends Field {
     return NumericUtils.sortableIntToFloat(numericValue().intValue());
   }
 
+  @Override
+  public StoredValue storedValue() {
+    return storedValue;
+  }
+
   @Override
   public String toString() {
     return getClass().getSimpleName() + " <" + name + ':' + getValueAsFloat() + '>';
@@ -85,6 +117,9 @@ public final class FloatField extends Field {
   @Override
   public void setFloatValue(float value) {
     super.setLongValue(NumericUtils.floatToSortableInt(value));
+    if (storedValue != null) {
+      storedValue.setFloatValue(value);
+    }
   }
 
   @Override
diff --git a/lucene/core/src/java/org/apache/lucene/document/IntField.java b/lucene/core/src/java/org/apache/lucene/document/IntField.java
index 30a9fb64be4..9d2ffd3c3f3 100644
--- a/lucene/core/src/java/org/apache/lucene/document/IntField.java
+++ b/lucene/core/src/java/org/apache/lucene/document/IntField.java
@@ -30,9 +30,9 @@ import org.apache.lucene.util.NumericUtils;
 
 /**
  * Field that stores a per-document <code>int</code> value for scoring, sorting or value retrieval
- * and index the field for fast range filters. If you also need to store the value, you should add a
- * separate {@link StoredField} instance. If you need more fine-grained control you can use {@link
- * IntPoint} and {@link NumericDocValuesField} or {@link SortedNumericDocValuesField}.
+ * and index the field for fast range filters. If you need more fine-grained control you can use
+ * {@link IntPoint}, {@link NumericDocValuesField} or {@link SortedNumericDocValuesField}, and
+ * {@link StoredField}.
  *
  * <p>This field defines static factory methods for creating common queries:
  *
@@ -47,23 +47,51 @@ import org.apache.lucene.util.NumericUtils;
 public final class IntField extends Field {
 
   private static final FieldType FIELD_TYPE = new FieldType();
+  private static final FieldType FIELD_TYPE_STORED;
 
   static {
     FIELD_TYPE.setDimensions(1, Integer.BYTES);
     FIELD_TYPE.setDocValuesType(DocValuesType.SORTED_NUMERIC);
     FIELD_TYPE.freeze();
+
+    FIELD_TYPE_STORED = new FieldType(FIELD_TYPE);
+    FIELD_TYPE_STORED.setStored(true);
+    FIELD_TYPE_STORED.freeze();
   }
 
+  private final StoredValue storedValue;
+
   /**
-   * Creates a new IntField, indexing the provided point and storing it as a DocValue
+   * Creates a new IntField, indexing the provided point and storing it as a DocValue.
    *
    * @param name field name
    * @param value the int value
    * @throws IllegalArgumentException if the field name or value is null.
+   * @deprecated Use {@link #IntField(String, int, Field.Store)} with {@link Field.Store#NO}
+   *     instead.
    */
+  @Deprecated
   public IntField(String name, int value) {
-    super(name, FIELD_TYPE);
+    this(name, value, Field.Store.NO);
+  }
+
+  /**
+   * Creates a new IntField, indexing the provided point, storing it as a DocValue, and optionally
+   * storing it as a stored field.
+   *
+   * @param name field name
+   * @param value the int value
+   * @param stored whether to store the field
+   * @throws IllegalArgumentException if the field name or value is null.
+   */
+  public IntField(String name, int value, Field.Store stored) {
+    super(name, stored == Field.Store.YES ? FIELD_TYPE_STORED : FIELD_TYPE);
     fieldsData = value;
+    if (stored == Field.Store.YES) {
+      storedValue = new StoredValue(value);
+    } else {
+      storedValue = null;
+    }
   }
 
   @Override
@@ -73,6 +101,19 @@ public final class IntField extends Field {
     return new BytesRef(bytes);
   }
 
+  @Override
+  public StoredValue storedValue() {
+    return storedValue;
+  }
+
+  @Override
+  public void setIntValue(int value) {
+    super.setIntValue(value);
+    if (storedValue != null) {
+      storedValue.setIntValue(value);
+    }
+  }
+
   @Override
   public String toString() {
     return getClass().getSimpleName() + " <" + name + ':' + fieldsData + '>';
diff --git a/lucene/core/src/java/org/apache/lucene/document/LongField.java b/lucene/core/src/java/org/apache/lucene/document/LongField.java
index af4dfd4bc81..aeb6f3ec6f6 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LongField.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LongField.java
@@ -33,9 +33,9 @@ import org.apache.lucene.util.NumericUtils;
 
 /**
  * Field that stores a per-document <code>long</code> value for scoring, sorting or value retrieval
- * and index the field for fast range filters. If you also need to store the value, you should add a
- * separate {@link StoredField} instance. If you need more fine-grained control you can use {@link
- * LongPoint} and {@link NumericDocValuesField} or {@link SortedNumericDocValuesField}.
+ * and index the field for fast range filters. If you need more fine-grained control you can use
+ * {@link LongPoint}, {@link NumericDocValuesField} or {@link SortedNumericDocValuesField}, and
+ * {@link StoredField}.
  *
  * <p>This field defines static factory methods for creating common queries:
  *
@@ -50,23 +50,51 @@ import org.apache.lucene.util.NumericUtils;
 public final class LongField extends Field {
 
   private static final FieldType FIELD_TYPE = new FieldType();
+  private static final FieldType FIELD_TYPE_STORED;
 
   static {
     FIELD_TYPE.setDimensions(1, Long.BYTES);
     FIELD_TYPE.setDocValuesType(DocValuesType.SORTED_NUMERIC);
     FIELD_TYPE.freeze();
+
+    FIELD_TYPE_STORED = new FieldType(FIELD_TYPE);
+    FIELD_TYPE_STORED.setStored(true);
+    FIELD_TYPE_STORED.freeze();
   }
 
+  private final StoredValue storedValue;
+
   /**
    * Creates a new LongField, indexing the provided point and storing it as a DocValue
    *
    * @param name field name
    * @param value the long value
    * @throws IllegalArgumentException if the field name or value is null.
+   * @deprecated Use {@link #LongField(String, int, Field.Store)} with {@link Field.Store#NO}
+   *     instead.
    */
+  @Deprecated
   public LongField(String name, long value) {
-    super(name, FIELD_TYPE);
+    this(name, value, Field.Store.NO);
+  }
+
+  /**
+   * Creates a new LongField, indexing the provided point, storing it as a DocValue, and optionally
+   * storing it as a stored field.
+   *
+   * @param name field name
+   * @param value the long value
+   * @param stored whether to store the field
+   * @throws IllegalArgumentException if the field name or value is null.
+   */
+  public LongField(String name, long value, Field.Store stored) {
+    super(name, stored == Field.Store.YES ? FIELD_TYPE_STORED : FIELD_TYPE);
     fieldsData = value;
+    if (stored == Field.Store.YES) {
+      storedValue = new StoredValue(value);
+    } else {
+      storedValue = null;
+    }
   }
 
   @Override
@@ -76,6 +104,19 @@ public final class LongField extends Field {
     return new BytesRef(bytes);
   }
 
+  @Override
+  public StoredValue storedValue() {
+    return storedValue;
+  }
+
+  @Override
+  public void setLongValue(long value) {
+    super.setLongValue(value);
+    if (storedValue != null) {
+      storedValue.setLongValue(value);
+    }
+  }
+
   @Override
   public String toString() {
     return getClass().getSimpleName() + " <" + name + ':' + fieldsData + '>';
diff --git a/lucene/core/src/java/org/apache/lucene/document/StoredValue.java b/lucene/core/src/java/org/apache/lucene/document/StoredValue.java
new file mode 100644
index 00000000000..61401c7b2fc
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/StoredValue.java
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.document;
+
+import java.util.Objects;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Abstraction around a stored value.
+ *
+ * @see IndexableField
+ */
+public final class StoredValue {
+
+  /** Type of a {@link StoredValue}. */
+  public enum Type {
+    /** Type of integer values. */
+    INTEGER,
+    /** Type of long values. */
+    LONG,
+    /** Type of float values. */
+    FLOAT,
+    /** Type of double values. */
+    DOUBLE,
+    /** Type of binary values. */
+    BINARY,
+    /** Type of string values. */
+    STRING;
+  }
+
+  private final Type type;
+  private int intValue;
+  private long longValue;
+  private float floatValue;
+  private double doubleValue;
+  private BytesRef binaryValue;
+  private String stringValue;
+
+  /** Ctor for integer values. */
+  public StoredValue(int value) {
+    type = Type.INTEGER;
+    intValue = value;
+  }
+
+  /** Ctor for long values. */
+  public StoredValue(long value) {
+    type = Type.LONG;
+    longValue = value;
+  }
+
+  /** Ctor for float values. */
+  public StoredValue(float value) {
+    type = Type.FLOAT;
+    floatValue = value;
+  }
+
+  /** Ctor for double values. */
+  public StoredValue(double value) {
+    type = Type.DOUBLE;
+    doubleValue = value;
+  }
+
+  /** Ctor for binary values. */
+  public StoredValue(BytesRef value) {
+    type = Type.BINARY;
+    binaryValue = Objects.requireNonNull(value);
+  }
+
+  /** Ctor for binary values. */
+  public StoredValue(String value) {
+    type = Type.STRING;
+    stringValue = Objects.requireNonNull(value);
+  }
+
+  /** Retrieve the type of the stored value. */
+  public Type getType() {
+    return type;
+  }
+
+  /** Set an integer value. */
+  public void setIntValue(int value) {
+    if (type != Type.INTEGER) {
+      throw new IllegalArgumentException("Cannot set an integer on a " + type + " value");
+    }
+    intValue = value;
+  }
+
+  /** Set a long value. */
+  public void setLongValue(long value) {
+    if (type != Type.LONG) {
+      throw new IllegalArgumentException("Cannot set a long on a " + type + " value");
+    }
+    longValue = value;
+  }
+
+  /** Set a float value. */
+  public void setFloatValue(float value) {
+    if (type != Type.FLOAT) {
+      throw new IllegalArgumentException("Cannot set a float on a " + type + " value");
+    }
+    floatValue = value;
+  }
+
+  /** Set a double value. */
+  public void setDoubleValue(double value) {
+    if (type != Type.DOUBLE) {
+      throw new IllegalArgumentException("Cannot set a double on a " + type + " value");
+    }
+    doubleValue = value;
+  }
+
+  /** Set a binary value. */
+  public void setBinaryValue(BytesRef value) {
+    if (type != Type.BINARY) {
+      throw new IllegalArgumentException("Cannot set a binary value on a " + type + " value");
+    }
+    binaryValue = Objects.requireNonNull(value);
+  }
+
+  /** Set a string value. */
+  public void setStringValue(String value) {
+    if (type != Type.STRING) {
+      throw new IllegalArgumentException("Cannot set a string value on a " + type + " value");
+    }
+    stringValue = Objects.requireNonNull(value);
+  }
+
+  /** Retrieve an integer value. */
+  public int getIntValue() {
+    if (type != Type.INTEGER) {
+      throw new IllegalArgumentException("Cannot get an integer on a " + type + " value");
+    }
+    return intValue;
+  }
+
+  /** Retrieve a long value. */
+  public long getLongValue() {
+    if (type != Type.LONG) {
+      throw new IllegalArgumentException("Cannot get a long on a " + type + " value");
+    }
+    return longValue;
+  }
+
+  /** Retrieve a float value. */
+  public float getFloatValue() {
+    if (type != Type.FLOAT) {
+      throw new IllegalArgumentException("Cannot get a float on a " + type + " value");
+    }
+    return floatValue;
+  }
+
+  /** Retrieve a double value. */
+  public double getDoubleValue() {
+    if (type != Type.DOUBLE) {
+      throw new IllegalArgumentException("Cannot get a double on a " + type + " value");
+    }
+    return doubleValue;
+  }
+
+  /** Retrieve a binary value. */
+  public BytesRef getBinaryValue() {
+    if (type != Type.BINARY) {
+      throw new IllegalArgumentException("Cannot get a binary value on a " + type + " value");
+    }
+    return binaryValue;
+  }
+
+  /** Retrieve a string value. */
+  public String getStringValue() {
+    if (type != Type.STRING) {
+      throw new IllegalArgumentException("Cannot get a string value on a " + type + " value");
+    }
+    return stringValue;
+  }
+}
diff --git a/lucene/core/src/java/org/apache/lucene/document/StringField.java b/lucene/core/src/java/org/apache/lucene/document/StringField.java
index 4e3ccebfa99..fb242346258 100644
--- a/lucene/core/src/java/org/apache/lucene/document/StringField.java
+++ b/lucene/core/src/java/org/apache/lucene/document/StringField.java
@@ -45,6 +45,8 @@ public final class StringField extends Field {
     TYPE_STORED.freeze();
   }
 
+  private final StoredValue storedValue;
+
   /**
    * Creates a new textual StringField, indexing the provided String value as a single token.
    *
@@ -55,6 +57,11 @@ public final class StringField extends Field {
    */
   public StringField(String name, String value, Store stored) {
     super(name, value, stored == Store.YES ? TYPE_STORED : TYPE_NOT_STORED);
+    if (stored == Store.YES) {
+      storedValue = new StoredValue(value);
+    } else {
+      storedValue = null;
+    }
   }
 
   /**
@@ -69,5 +76,31 @@ public final class StringField extends Field {
    */
   public StringField(String name, BytesRef value, Store stored) {
     super(name, value, stored == Store.YES ? TYPE_STORED : TYPE_NOT_STORED);
+    if (stored == Store.YES) {
+      storedValue = new StoredValue(value);
+    } else {
+      storedValue = null;
+    }
+  }
+
+  @Override
+  public void setStringValue(String value) {
+    super.setStringValue(value);
+    if (storedValue != null) {
+      storedValue.setStringValue(value);
+    }
+  }
+
+  @Override
+  public void setBytesValue(BytesRef value) {
+    super.setBytesValue(value);
+    if (storedValue != null) {
+      storedValue.setBinaryValue(value);
+    }
+  }
+
+  @Override
+  public StoredValue storedValue() {
+    return storedValue;
   }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/document/TextField.java b/lucene/core/src/java/org/apache/lucene/document/TextField.java
index 409aa7a55c9..b6cb6e1fa23 100644
--- a/lucene/core/src/java/org/apache/lucene/document/TextField.java
+++ b/lucene/core/src/java/org/apache/lucene/document/TextField.java
@@ -45,6 +45,8 @@ public final class TextField extends Field {
 
   // TODO: add sugar for term vectors...?
 
+  private final StoredValue storedValue;
+
   /**
    * Creates a new un-stored TextField with Reader value.
    *
@@ -55,6 +57,7 @@ public final class TextField extends Field {
    */
   public TextField(String name, Reader reader) {
     super(name, reader, TYPE_NOT_STORED);
+    storedValue = null;
   }
 
   /**
@@ -67,6 +70,11 @@ public final class TextField extends Field {
    */
   public TextField(String name, String value, Store store) {
     super(name, value, store == Store.YES ? TYPE_STORED : TYPE_NOT_STORED);
+    if (store == Store.YES) {
+      storedValue = new StoredValue(value);
+    } else {
+      storedValue = null;
+    }
   }
 
   /**
@@ -79,5 +87,19 @@ public final class TextField extends Field {
    */
   public TextField(String name, TokenStream stream) {
     super(name, stream, TYPE_NOT_STORED);
+    storedValue = null;
+  }
+
+  @Override
+  public void setStringValue(String value) {
+    super.setStringValue(value);
+    if (storedValue != null) {
+      storedValue.setStringValue(value);
+    }
+  }
+
+  @Override
+  public StoredValue storedValue() {
+    return storedValue;
   }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexableField.java b/lucene/core/src/java/org/apache/lucene/index/IndexableField.java
index 6cbc8437856..0715d327b83 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexableField.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexableField.java
@@ -19,6 +19,7 @@ package org.apache.lucene.index;
 import java.io.Reader;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.document.StoredValue;
 import org.apache.lucene.util.BytesRef;
 
 // TODO: how to handle versioning here...?
@@ -68,4 +69,10 @@ public interface IndexableField {
 
   /** Non-null if this field has a numeric value */
   public Number numericValue();
+
+  /**
+   * Stored value. This method is called to populate stored fields and must return a non-null value
+   * if the field stored.
+   */
+  public StoredValue storedValue();
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexingChain.java b/lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
index dd5555765d9..ebcf26bce26 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
@@ -40,6 +40,7 @@ import org.apache.lucene.codecs.PointsWriter;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.KnnByteVectorField;
 import org.apache.lucene.document.KnnFloatVectorField;
+import org.apache.lucene.document.StoredValue;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.search.SortField;
@@ -697,17 +698,20 @@ final class IndexingChain implements Accountable {
 
     // Add stored fields
     if (fieldType.stored()) {
-      String value = field.stringValue();
-      if (value != null && value.length() > IndexWriter.MAX_STORED_STRING_LENGTH) {
+      StoredValue storedValue = field.storedValue();
+      if (storedValue == null) {
+        throw new IllegalArgumentException("Cannot store a null value");
+      } else if (storedValue.getType() == StoredValue.Type.STRING
+          && storedValue.getStringValue().length() > IndexWriter.MAX_STORED_STRING_LENGTH) {
         throw new IllegalArgumentException(
             "stored field \""
                 + field.name()
                 + "\" is too large ("
-                + value.length()
+                + storedValue.getStringValue().length()
                 + " characters) to store");
       }
       try {
-        storedFieldsConsumer.writeField(pf.fieldInfo, field);
+        storedFieldsConsumer.writeField(pf.fieldInfo, storedValue);
       } catch (Throwable th) {
         onAbortingException(th);
         throw th;
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortingStoredFieldsConsumer.java b/lucene/core/src/java/org/apache/lucene/index/SortingStoredFieldsConsumer.java
index 0e3d16bd0c9..55a87641e86 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortingStoredFieldsConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortingStoredFieldsConsumer.java
@@ -18,10 +18,7 @@
 package org.apache.lucene.index;
 
 import java.io.IOException;
-import java.io.Reader;
 import java.util.Objects;
-import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.StoredFieldsFormat;
 import org.apache.lucene.codecs.StoredFieldsReader;
@@ -30,7 +27,6 @@ import org.apache.lucene.codecs.compressing.CompressionMode;
 import org.apache.lucene.codecs.compressing.Compressor;
 import org.apache.lucene.codecs.compressing.Decompressor;
 import org.apache.lucene.codecs.lucene90.compressing.Lucene90CompressingStoredFieldsFormat;
-import org.apache.lucene.document.StoredField;
 import org.apache.lucene.store.ByteBuffersDataInput;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
@@ -136,12 +132,8 @@ final class SortingStoredFieldsConsumer extends StoredFieldsConsumer {
   }
 
   /** A visitor that copies every field it sees in the provided {@link StoredFieldsWriter}. */
-  private static class CopyVisitor extends StoredFieldVisitor implements IndexableField {
+  private static class CopyVisitor extends StoredFieldVisitor {
     final StoredFieldsWriter writer;
-    BytesRef binaryValue;
-    String stringValue;
-    Number numericValue;
-    FieldInfo currentField;
 
     CopyVisitor(StoredFieldsWriter writer) {
       this.writer = writer;
@@ -149,96 +141,39 @@ final class SortingStoredFieldsConsumer extends StoredFieldsConsumer {
 
     @Override
     public void binaryField(FieldInfo fieldInfo, byte[] value) throws IOException {
-      reset(fieldInfo);
       // TODO: can we avoid new BR here?
-      binaryValue = new BytesRef(value);
-      write();
+      writer.writeField(fieldInfo, new BytesRef(value));
     }
 
     @Override
     public void stringField(FieldInfo fieldInfo, String value) throws IOException {
-      reset(fieldInfo);
-      stringValue = Objects.requireNonNull(value, "String value should not be null");
-      write();
+      writer.writeField(
+          fieldInfo, Objects.requireNonNull(value, "String value should not be null"));
     }
 
     @Override
     public void intField(FieldInfo fieldInfo, int value) throws IOException {
-      reset(fieldInfo);
-      numericValue = value;
-      write();
+      writer.writeField(fieldInfo, value);
     }
 
     @Override
     public void longField(FieldInfo fieldInfo, long value) throws IOException {
-      reset(fieldInfo);
-      numericValue = value;
-      write();
+      writer.writeField(fieldInfo, value);
     }
 
     @Override
     public void floatField(FieldInfo fieldInfo, float value) throws IOException {
-      reset(fieldInfo);
-      numericValue = value;
-      write();
+      writer.writeField(fieldInfo, value);
     }
 
     @Override
     public void doubleField(FieldInfo fieldInfo, double value) throws IOException {
-      reset(fieldInfo);
-      numericValue = value;
-      write();
+      writer.writeField(fieldInfo, value);
     }
 
     @Override
     public Status needsField(FieldInfo fieldInfo) throws IOException {
       return Status.YES;
     }
-
-    @Override
-    public String name() {
-      return currentField.name;
-    }
-
-    @Override
-    public IndexableFieldType fieldType() {
-      return StoredField.TYPE;
-    }
-
-    @Override
-    public BytesRef binaryValue() {
-      return binaryValue;
-    }
-
-    @Override
-    public String stringValue() {
-      return stringValue;
-    }
-
-    @Override
-    public Number numericValue() {
-      return numericValue;
-    }
-
-    @Override
-    public Reader readerValue() {
-      return null;
-    }
-
-    @Override
-    public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) {
-      return null;
-    }
-
-    void reset(FieldInfo field) {
-      currentField = field;
-      binaryValue = null;
-      stringValue = null;
-      numericValue = null;
-    }
-
-    void write() throws IOException {
-      writer.writeField(currentField, this);
-    }
   }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java b/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java
index 26157e74344..1bde9d4d597 100644
--- a/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.StoredFieldsWriter;
+import org.apache.lucene.document.StoredValue;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.Accountable;
@@ -61,8 +62,29 @@ class StoredFieldsConsumer {
     writer.startDocument();
   }
 
-  void writeField(FieldInfo info, IndexableField field) throws IOException {
-    writer.writeField(info, field);
+  void writeField(FieldInfo info, StoredValue value) throws IOException {
+    switch (value.getType()) {
+      case INTEGER:
+        writer.writeField(info, value.getIntValue());
+        break;
+      case LONG:
+        writer.writeField(info, value.getLongValue());
+        break;
+      case FLOAT:
+        writer.writeField(info, value.getFloatValue());
+        break;
+      case DOUBLE:
+        writer.writeField(info, value.getDoubleValue());
+        break;
+      case BINARY:
+        writer.writeField(info, value.getBinaryValue());
+        break;
+      case STRING:
+        writer.writeField(info, value.getStringValue());
+        break;
+      default:
+        throw new AssertionError();
+    }
   }
 
   void finishDocument() throws IOException {
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/compressing/TestCompressingStoredFieldsFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/compressing/TestCompressingStoredFieldsFormat.java
index c1096b71409..9c6140801a9 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene90/compressing/TestCompressingStoredFieldsFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene90/compressing/TestCompressingStoredFieldsFormat.java
@@ -16,14 +16,10 @@
  */
 package org.apache.lucene.codecs.lucene90.compressing;
 
-import com.carrotsearch.randomizedtesting.generators.RandomNumbers;
 import java.io.IOException;
 import java.util.Random;
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.IntPoint;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.CodecReader;
 import org.apache.lucene.index.DirectoryReader;
@@ -53,51 +49,6 @@ public class TestCompressingStoredFieldsFormat extends BaseStoredFieldsFormatTes
     }
   }
 
-  public void testDeletePartiallyWrittenFilesIfAbort() throws IOException {
-    Directory dir = newDirectory();
-    IndexWriterConfig iwConf = newIndexWriterConfig(new MockAnalyzer(random()));
-    iwConf.setMaxBufferedDocs(RandomNumbers.randomIntBetween(random(), 2, 30));
-    iwConf.setCodec(getCodec());
-    // disable CFS because this test checks file names
-    iwConf.setMergePolicy(newLogMergePolicy(false));
-    iwConf.setUseCompoundFile(false);
-
-    // Cannot use RIW because this test wants CFS to stay off:
-    IndexWriter iw = new IndexWriter(dir, iwConf);
-
-    final Document validDoc = new Document();
-    validDoc.add(new IntPoint("id", 0));
-    validDoc.add(new StoredField("id", 0));
-    iw.addDocument(validDoc);
-    iw.commit();
-
-    // make sure that #writeField will fail to trigger an abort
-    final Document invalidDoc = new Document();
-    FieldType fieldType = new FieldType();
-    fieldType.setStored(true);
-    invalidDoc.add(
-        new Field("invalid", fieldType) {
-
-          @Override
-          public String stringValue() {
-            // TODO: really bad & scary that this causes IW to
-            // abort the segment!!  We should fix this.
-            return null;
-          }
-        });
-
-    try {
-      iw.addDocument(invalidDoc);
-      iw.commit();
-    } catch (IllegalArgumentException iae) {
-      // expected
-      assertEquals(iae, iw.getTragicException());
-    }
-    // Writer should be closed by tragedy
-    assertFalse(iw.isOpen());
-    dir.close();
-  }
-
   public void testZFloat() throws Exception {
     byte[] buffer = new byte[5]; // we never need more than 5 bytes
     ByteArrayDataOutput out = new ByteArrayDataOutput(buffer);
diff --git a/lucene/core/src/test/org/apache/lucene/document/TestField.java b/lucene/core/src/test/org/apache/lucene/document/TestField.java
index 064c8337982..e813721b595 100644
--- a/lucene/core/src/test/org/apache/lucene/document/TestField.java
+++ b/lucene/core/src/test/org/apache/lucene/document/TestField.java
@@ -215,92 +215,135 @@ public class TestField extends LuceneTestCase {
   }
 
   public void testIntField() throws Exception {
-    IntField field = new IntField("foo", 12);
+    Field[] fields =
+        new Field[] {
+          new IntField("foo", 12, Field.Store.NO), new IntField("foo", 12, Field.Store.YES),
+        };
 
-    trySetByteValue(field);
-    trySetBytesValue(field);
-    trySetBytesRefValue(field);
-    trySetDoubleValue(field);
-    field.setIntValue(6);
-    trySetLongValue(field);
-    trySetFloatValue(field);
-    trySetLongValue(field);
-    trySetReaderValue(field);
-    trySetShortValue(field);
-    trySetStringValue(field);
-    trySetTokenStreamValue(field);
+    for (Field field : fields) {
+      trySetByteValue(field);
+      trySetBytesValue(field);
+      trySetBytesRefValue(field);
+      trySetDoubleValue(field);
+      field.setIntValue(6);
+      trySetLongValue(field);
+      trySetFloatValue(field);
+      trySetLongValue(field);
+      trySetReaderValue(field);
+      trySetShortValue(field);
+      trySetStringValue(field);
+      trySetTokenStreamValue(field);
 
-    assertEquals(6, field.numericValue().intValue());
-    assertEquals(6, NumericUtils.sortableBytesToInt(field.binaryValue().bytes, 0));
-    assertEquals("IntField <foo:6>", field.toString());
+      assertEquals(6, field.numericValue().intValue());
+      assertEquals(6, NumericUtils.sortableBytesToInt(field.binaryValue().bytes, 0));
+      assertEquals("IntField <foo:6>", field.toString());
+      if (field.fieldType().stored()) {
+        assertEquals(6, field.storedValue().getIntValue());
+      } else {
+        assertNull(field.storedValue());
+      }
+    }
   }
 
   public void testLongField() throws Exception {
-    LongField field = new LongField("foo", 12);
+    Field[] fields =
+        new Field[] {
+          new LongField("foo", 12, Field.Store.NO), new LongField("foo", 12, Field.Store.YES),
+        };
 
-    trySetByteValue(field);
-    trySetBytesValue(field);
-    trySetBytesRefValue(field);
-    trySetDoubleValue(field);
-    trySetIntValue(field);
-    field.setLongValue(6);
-    trySetFloatValue(field);
-    trySetReaderValue(field);
-    trySetShortValue(field);
-    trySetStringValue(field);
-    trySetTokenStreamValue(field);
+    for (Field field : fields) {
+      trySetByteValue(field);
+      trySetBytesValue(field);
+      trySetBytesRefValue(field);
+      trySetDoubleValue(field);
+      trySetIntValue(field);
+      field.setLongValue(6);
+      trySetFloatValue(field);
+      trySetReaderValue(field);
+      trySetShortValue(field);
+      trySetStringValue(field);
+      trySetTokenStreamValue(field);
 
-    assertEquals(6L, field.numericValue().longValue());
-    assertEquals(6L, NumericUtils.sortableBytesToLong(field.binaryValue().bytes, 0));
-    assertEquals("LongField <foo:6>", field.toString());
+      assertEquals(6L, field.numericValue().longValue());
+      assertEquals(6L, NumericUtils.sortableBytesToLong(field.binaryValue().bytes, 0));
+      assertEquals("LongField <foo:6>", field.toString());
+      if (field.fieldType().stored()) {
+        assertEquals(6, field.storedValue().getLongValue());
+      } else {
+        assertNull(field.storedValue());
+      }
+    }
   }
 
   public void testFloatField() throws Exception {
-    FloatField field = new FloatField("foo", 12.6f);
+    Field[] fields =
+        new Field[] {
+          new FloatField("foo", 12.6f, Field.Store.NO),
+          new FloatField("foo", 12.6f, Field.Store.YES),
+        };
 
-    assertEquals(12.6f, NumericUtils.sortableIntToFloat(field.numericValue().intValue()), 0.0f);
-    assertEquals(12.6f, FloatPoint.decodeDimension(field.binaryValue().bytes, 0), 0.0f);
-    assertEquals("FloatField <foo:12.6>", field.toString());
+    for (Field field : fields) {
+      assertEquals(12.6f, NumericUtils.sortableIntToFloat(field.numericValue().intValue()), 0.0f);
+      assertEquals(12.6f, FloatPoint.decodeDimension(field.binaryValue().bytes, 0), 0.0f);
+      assertEquals("FloatField <foo:12.6>", field.toString());
 
-    trySetByteValue(field);
-    trySetBytesValue(field);
-    trySetBytesRefValue(field);
-    trySetDoubleValue(field);
-    trySetIntValue(field);
-    trySetLongValue(field);
-    field.setFloatValue(-28.8f);
-    trySetReaderValue(field);
-    trySetShortValue(field);
-    trySetStringValue(field);
-    trySetTokenStreamValue(field);
+      trySetByteValue(field);
+      trySetBytesValue(field);
+      trySetBytesRefValue(field);
+      trySetDoubleValue(field);
+      trySetIntValue(field);
+      trySetLongValue(field);
+      field.setFloatValue(-28.8f);
+      trySetReaderValue(field);
+      trySetShortValue(field);
+      trySetStringValue(field);
+      trySetTokenStreamValue(field);
 
-    assertEquals(-28.8f, NumericUtils.sortableIntToFloat(field.numericValue().intValue()), 0.0f);
-    assertEquals(-28.8f, FloatPoint.decodeDimension(field.binaryValue().bytes, 0), 0.0f);
-    assertEquals("FloatField <foo:-28.8>", field.toString());
+      assertEquals(-28.8f, NumericUtils.sortableIntToFloat(field.numericValue().intValue()), 0.0f);
+      assertEquals(-28.8f, FloatPoint.decodeDimension(field.binaryValue().bytes, 0), 0.0f);
+      assertEquals("FloatField <foo:-28.8>", field.toString());
+      if (field.fieldType().stored()) {
+        assertEquals(-28.8f, field.storedValue().getFloatValue(), 0f);
+      } else {
+        assertNull(field.storedValue());
+      }
+    }
   }
 
   public void testDoubleField() throws Exception {
-    DoubleField field = new DoubleField("foo", 12.7);
+    Field[] fields =
+        new Field[] {
+          new DoubleField("foo", 12.7, Field.Store.NO),
+          new DoubleField("foo", 12.7, Field.Store.YES),
+        };
 
-    assertEquals(12.7, NumericUtils.sortableLongToDouble(field.numericValue().longValue()), 0.0f);
-    assertEquals(12.7, DoublePoint.decodeDimension(field.binaryValue().bytes, 0), 0.0f);
-    assertEquals("DoubleField <foo:12.7>", field.toString());
+    for (Field field : fields) {
+      assertEquals(12.7, NumericUtils.sortableLongToDouble(field.numericValue().longValue()), 0.0f);
+      assertEquals(12.7, DoublePoint.decodeDimension(field.binaryValue().bytes, 0), 0.0f);
+      assertEquals("DoubleField <foo:12.7>", field.toString());
 
-    trySetByteValue(field);
-    trySetBytesValue(field);
-    trySetBytesRefValue(field);
-    trySetIntValue(field);
-    trySetLongValue(field);
-    trySetFloatValue(field);
-    field.setDoubleValue(-28.8);
-    trySetReaderValue(field);
-    trySetShortValue(field);
-    trySetStringValue(field);
-    trySetTokenStreamValue(field);
+      trySetByteValue(field);
+      trySetBytesValue(field);
+      trySetBytesRefValue(field);
+      trySetIntValue(field);
+      trySetLongValue(field);
+      trySetFloatValue(field);
+      field.setDoubleValue(-28.8);
+      trySetReaderValue(field);
+      trySetShortValue(field);
+      trySetStringValue(field);
+      trySetTokenStreamValue(field);
 
-    assertEquals(-28.8, NumericUtils.sortableLongToDouble(field.numericValue().longValue()), 0.0f);
-    assertEquals(-28.8, DoublePoint.decodeDimension(field.binaryValue().bytes, 0), 0.0f);
-    assertEquals("DoubleField <foo:-28.8>", field.toString());
+      assertEquals(
+          -28.8, NumericUtils.sortableLongToDouble(field.numericValue().longValue()), 0.0f);
+      assertEquals(-28.8, DoublePoint.decodeDimension(field.binaryValue().bytes, 0), 0.0f);
+      assertEquals("DoubleField <foo:-28.8>", field.toString());
+      if (field.fieldType().stored()) {
+        assertEquals(-28.8, field.storedValue().getDoubleValue(), 0d);
+      } else {
+        assertNull(field.storedValue());
+      }
+    }
   }
 
   public void testNumericDocValuesField() throws Exception {
@@ -423,6 +466,11 @@ public class TestField extends LuceneTestCase {
       trySetTokenStreamValue(field);
 
       assertEquals("baz", field.stringValue());
+      if (field.fieldType().stored()) {
+        assertEquals("baz", field.storedValue().getStringValue());
+      } else {
+        assertNull(field.storedValue());
+      }
     }
   }
 
@@ -448,6 +496,11 @@ public class TestField extends LuceneTestCase {
       trySetTokenStreamValue(field);
 
       assertEquals(new BytesRef("baz"), field.binaryValue());
+      if (field.fieldType().stored()) {
+        assertEquals(new BytesRef("baz"), field.storedValue().getBinaryValue());
+      } else {
+        assertNull(field.storedValue());
+      }
     }
   }
 
@@ -471,6 +524,11 @@ public class TestField extends LuceneTestCase {
       trySetTokenStreamValue(field);
 
       assertEquals("baz", field.stringValue());
+      if (field.fieldType().stored()) {
+        assertEquals("baz", field.storedValue().getStringValue());
+      } else {
+        assertNull(field.storedValue());
+      }
     }
   }
 
@@ -490,6 +548,7 @@ public class TestField extends LuceneTestCase {
     trySetTokenStreamValue(field);
 
     assertNotNull(field.readerValue());
+    assertNull(field.storedValue());
   }
 
   /* TODO: this is pretty expert and crazy
diff --git a/lucene/core/src/test/org/apache/lucene/document/TestLongDistanceFeatureQuery.java b/lucene/core/src/test/org/apache/lucene/document/TestLongDistanceFeatureQuery.java
index 85641cb615d..2f35024c220 100644
--- a/lucene/core/src/test/org/apache/lucene/document/TestLongDistanceFeatureQuery.java
+++ b/lucene/core/src/test/org/apache/lucene/document/TestLongDistanceFeatureQuery.java
@@ -17,6 +17,7 @@
 package org.apache.lucene.document;
 
 import java.io.IOException;
+import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
@@ -61,7 +62,7 @@ public class TestLongDistanceFeatureQuery extends LuceneTestCase {
             dir,
             newIndexWriterConfig().setMergePolicy(newLogMergePolicy(random().nextBoolean())));
     Document doc = new Document();
-    LongField field = new LongField("foo", 0L);
+    LongField field = new LongField("foo", 0L, Store.NO);
     doc.add(field);
 
     field.setLongValue(3);
@@ -123,7 +124,7 @@ public class TestLongDistanceFeatureQuery extends LuceneTestCase {
             dir,
             newIndexWriterConfig().setMergePolicy(newLogMergePolicy(random().nextBoolean())));
     Document doc = new Document();
-    LongField field = new LongField("foo", 0L);
+    LongField field = new LongField("foo", 0L, Store.NO);
     doc.add(field);
 
     field.setLongValue(3);
@@ -209,7 +210,7 @@ public class TestLongDistanceFeatureQuery extends LuceneTestCase {
             dir,
             newIndexWriterConfig().setMergePolicy(newLogMergePolicy(random().nextBoolean())));
     Document doc = new Document();
-    LongField field = new LongField("foo", 0L);
+    LongField field = new LongField("foo", 0L, Store.NO);
     doc.add(field);
 
     field.setLongValue(3);
@@ -254,31 +255,31 @@ public class TestLongDistanceFeatureQuery extends LuceneTestCase {
 
     Document doc = new Document();
     for (long v : new long[] {3, 1000, Long.MAX_VALUE}) {
-      doc.add(new LongField("foo", v));
+      doc.add(new LongField("foo", v, Store.NO));
     }
     w.addDocument(doc);
 
     doc = new Document();
     for (long v : new long[] {-100, 12, 999}) {
-      doc.add(new LongField("foo", v));
+      doc.add(new LongField("foo", v, Store.NO));
     }
     w.addDocument(doc);
 
     doc = new Document();
     for (long v : new long[] {Long.MIN_VALUE, -1000, 8}) {
-      doc.add(new LongField("foo", v));
+      doc.add(new LongField("foo", v, Store.NO));
     }
     w.addDocument(doc);
 
     doc = new Document();
     for (long v : new long[] {-1}) {
-      doc.add(new LongField("foo", v));
+      doc.add(new LongField("foo", v, Store.NO));
     }
     w.addDocument(doc);
 
     doc = new Document();
     for (long v : new long[] {Long.MIN_VALUE, 7}) {
-      doc.add(new LongField("foo", v));
+      doc.add(new LongField("foo", v, Store.NO));
     }
     w.addDocument(doc);
 
@@ -324,7 +325,7 @@ public class TestLongDistanceFeatureQuery extends LuceneTestCase {
         new IndexWriter(
             dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy(random().nextBoolean())));
     Document doc = new Document();
-    LongField field = new LongField("foo", 0L);
+    LongField field = new LongField("foo", 0L, Store.NO);
     doc.add(field);
 
     int numDocs = atLeast(10000);
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestFieldReuse.java b/lucene/core/src/test/org/apache/lucene/index/TestFieldReuse.java
index a1c8bd89f91..1a81651800b 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestFieldReuse.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestFieldReuse.java
@@ -22,6 +22,7 @@ import java.util.Collections;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.StoredValue;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.tests.analysis.BaseTokenStreamTestCase;
@@ -92,6 +93,11 @@ public class TestFieldReuse extends BaseTokenStreamTestCase {
     public Number numericValue() {
       return null;
     }
+
+    @Override
+    public StoredValue storedValue() {
+      return null;
+    }
   }
 
   public void testIndexWriterActuallyReuses() throws IOException {
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java
index 852e1ea1c89..d5f52338519 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java
@@ -27,6 +27,7 @@ import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.StoredField;
+import org.apache.lucene.document.StoredValue;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -186,6 +187,17 @@ public class TestIndexableField extends LuceneTestCase {
           ? analyzer.tokenStream(name(), readerValue())
           : analyzer.tokenStream(name(), new StringReader(stringValue()));
     }
+
+    @Override
+    public StoredValue storedValue() {
+      if (stringValue() != null) {
+        return new StoredValue(stringValue());
+      } else if (binaryValue() != null) {
+        return new StoredValue(binaryValue());
+      } else {
+        return null;
+      }
+    }
   }
 
   // Silly test showing how to index documents w/o using Lucene's core
@@ -388,6 +400,11 @@ public class TestIndexableField extends LuceneTestCase {
       ft.freeze();
       return ft;
     }
+
+    @Override
+    public StoredValue storedValue() {
+      return null;
+    }
   }
 
   // LUCENE-5611
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestIndexOrDocValuesQuery.java b/lucene/core/src/test/org/apache/lucene/search/TestIndexOrDocValuesQuery.java
index b8fe90581eb..19d562db795 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestIndexOrDocValuesQuery.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestIndexOrDocValuesQuery.java
@@ -114,15 +114,15 @@ public class TestIndexOrDocValuesQuery extends LuceneTestCase {
       if (i < 1000) {
         doc.add(new StringField("f1", "bar", Store.NO));
         for (int j = 0; j < 500; j++) {
-          doc.add(new LongField("f2", 42L));
+          doc.add(new LongField("f2", 42L, Store.NO));
         }
       } else if (i == 1001) {
         doc.add(new StringField("f1", "foo", Store.NO));
-        doc.add(new LongField("f2", 2L));
+        doc.add(new LongField("f2", 2L, Store.NO));
       } else {
         doc.add(new StringField("f1", "bar", Store.NO));
         for (int j = 0; j < 100; j++) {
-          doc.add(new LongField("f2", 2L));
+          doc.add(new LongField("f2", 2L, Store.NO));
         }
       }
       w.addDocument(doc);
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java b/lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java
index 7041fbb95a4..5c0aad74d97 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java
@@ -809,8 +809,8 @@ public class TestSortOptimization extends LuceneTestCase {
       int value = random().nextInt();
       int value2 = random().nextInt();
       final Document doc = new Document();
-      doc.add(new LongField("my_field", value));
-      doc.add(new LongField("my_field", value2));
+      doc.add(new LongField("my_field", value, Store.NO));
+      doc.add(new LongField("my_field", value2, Store.NO));
       writer.addDocument(doc);
     }
     final IndexReader reader = DirectoryReader.open(writer);
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSortedNumericSortField.java b/lucene/core/src/test/org/apache/lucene/search/TestSortedNumericSortField.java
index 3929010441b..d6919f014dd 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestSortedNumericSortField.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestSortedNumericSortField.java
@@ -19,6 +19,7 @@ package org.apache.lucene.search;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DoubleField;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.FloatField;
 import org.apache.lucene.document.IntField;
 import org.apache.lucene.index.IndexReader;
@@ -80,12 +81,12 @@ public class TestSortedNumericSortField extends LuceneTestCase {
     Directory dir = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
     Document doc = new Document();
-    doc.add(new IntField("value", 5));
+    doc.add(new IntField("value", 5, Store.NO));
     doc.add(newStringField("id", "2", Field.Store.YES));
     writer.addDocument(doc);
     doc = new Document();
-    doc.add(new IntField("value", 3));
-    doc.add(new IntField("value", 7));
+    doc.add(new IntField("value", 3, Store.NO));
+    doc.add(new IntField("value", 7, Store.NO));
     doc.add(newStringField("id", "1", Field.Store.YES));
     writer.addDocument(doc);
     IndexReader ir = writer.getReader();
@@ -108,12 +109,12 @@ public class TestSortedNumericSortField extends LuceneTestCase {
     Directory dir = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
     Document doc = new Document();
-    doc.add(new IntField("value", 3));
-    doc.add(new IntField("value", 7));
+    doc.add(new IntField("value", 3, Store.NO));
+    doc.add(new IntField("value", 7, Store.NO));
     doc.add(newStringField("id", "1", Field.Store.YES));
     writer.addDocument(doc);
     doc = new Document();
-    doc.add(new IntField("value", 5));
+    doc.add(new IntField("value", 5, Store.NO));
     doc.add(newStringField("id", "2", Field.Store.YES));
     writer.addDocument(doc);
 
@@ -137,12 +138,12 @@ public class TestSortedNumericSortField extends LuceneTestCase {
     Directory dir = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
     Document doc = new Document();
-    doc.add(new IntField("value", 5));
+    doc.add(new IntField("value", 5, Store.NO));
     doc.add(newStringField("id", "2", Field.Store.YES));
     writer.addDocument(doc);
     doc = new Document();
-    doc.add(new IntField("value", 3));
-    doc.add(new IntField("value", 7));
+    doc.add(new IntField("value", 3, Store.NO));
+    doc.add(new IntField("value", 7, Store.NO));
     doc.add(newStringField("id", "1", Field.Store.YES));
     writer.addDocument(doc);
     doc = new Document();
@@ -172,12 +173,12 @@ public class TestSortedNumericSortField extends LuceneTestCase {
     Directory dir = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
     Document doc = new Document();
-    doc.add(new IntField("value", 5));
+    doc.add(new IntField("value", 5, Store.NO));
     doc.add(newStringField("id", "2", Field.Store.YES));
     writer.addDocument(doc);
     doc = new Document();
-    doc.add(new IntField("value", 3));
-    doc.add(new IntField("value", 7));
+    doc.add(new IntField("value", 3, Store.NO));
+    doc.add(new IntField("value", 7, Store.NO));
     doc.add(newStringField("id", "1", Field.Store.YES));
     writer.addDocument(doc);
     doc = new Document();
@@ -207,11 +208,11 @@ public class TestSortedNumericSortField extends LuceneTestCase {
     Directory dir = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
     Document doc = new Document();
-    doc.add(new IntField("value", 5));
+    doc.add(new IntField("value", 5, Store.NO));
     doc.add(newStringField("id", "2", Field.Store.YES));
     writer.addDocument(doc);
     doc = new Document();
-    doc.add(new IntField("value", 3));
+    doc.add(new IntField("value", 3, Store.NO));
     doc.add(newStringField("id", "1", Field.Store.YES));
     writer.addDocument(doc);
     IndexReader ir = writer.getReader();
@@ -234,12 +235,12 @@ public class TestSortedNumericSortField extends LuceneTestCase {
     Directory dir = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
     Document doc = new Document();
-    doc.add(new FloatField("value", -3f));
+    doc.add(new FloatField("value", -3f, Store.NO));
     doc.add(newStringField("id", "2", Field.Store.YES));
     writer.addDocument(doc);
     doc = new Document();
-    doc.add(new FloatField("value", -5f));
-    doc.add(new FloatField("value", 7f));
+    doc.add(new FloatField("value", -5f, Store.NO));
+    doc.add(new FloatField("value", 7f, Store.NO));
     doc.add(newStringField("id", "1", Field.Store.YES));
     writer.addDocument(doc);
     IndexReader ir = writer.getReader();
@@ -262,12 +263,12 @@ public class TestSortedNumericSortField extends LuceneTestCase {
     Directory dir = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
     Document doc = new Document();
-    doc.add(new DoubleField("value", -3d));
+    doc.add(new DoubleField("value", -3d, Field.Store.YES));
     doc.add(newStringField("id", "2", Field.Store.YES));
     writer.addDocument(doc);
     doc = new Document();
-    doc.add(new DoubleField("value", -5d));
-    doc.add(new DoubleField("value", 7d));
+    doc.add(new DoubleField("value", -5d, Field.Store.YES));
+    doc.add(new DoubleField("value", 7d, Field.Store.YES));
     doc.add(newStringField("id", "1", Field.Store.YES));
     writer.addDocument(doc);
     IndexReader ir = writer.getReader();
diff --git a/lucene/demo/src/java/org/apache/lucene/demo/IndexFiles.java b/lucene/demo/src/java/org/apache/lucene/demo/IndexFiles.java
index 4120bb4fea6..7b172d65d41 100644
--- a/lucene/demo/src/java/org/apache/lucene/demo/IndexFiles.java
+++ b/lucene/demo/src/java/org/apache/lucene/demo/IndexFiles.java
@@ -245,7 +245,7 @@ public class IndexFiles implements AutoCloseable {
       // year/month/day/hour/minutes/seconds, down the resolution you require.
       // For example the long value 2011021714 would mean
       // February 17, 2011, 2-3 PM.
-      doc.add(new LongField("modified", lastModified));
+      doc.add(new LongField("modified", lastModified, Field.Store.NO));
 
       // Add the contents of the file to a field named "contents".  Specify a Reader,
       // so that the text of the file is tokenized and indexed, but not stored.
diff --git a/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java b/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
index 8ac20164103..6f87e080ee9 100644
--- a/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
+++ b/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
@@ -37,6 +37,7 @@ import org.apache.lucene.document.BinaryPoint;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DoublePoint;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.FloatPoint;
 import org.apache.lucene.document.IntField;
@@ -818,7 +819,7 @@ public class TestMemoryIndex extends LuceneTestCase {
           }
         };
 
-    Field intField = new IntField("int_field", 50);
+    Field intField = new IntField("int_field", 50, Store.NO);
 
     MemoryIndex index = MemoryIndex.fromDocument(Arrays.asList(field, multiField, intField), null);
     IndexSearcher searcher = index.createSearcher();
diff --git a/lucene/misc/src/java/org/apache/lucene/misc/document/LazyDocument.java b/lucene/misc/src/java/org/apache/lucene/misc/document/LazyDocument.java
index 63a1d1019fd..581eb90785c 100644
--- a/lucene/misc/src/java/org/apache/lucene/misc/document/LazyDocument.java
+++ b/lucene/misc/src/java/org/apache/lucene/misc/document/LazyDocument.java
@@ -27,6 +27,7 @@ import java.util.Set;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.document.Document;
+import org.apache.lucene.document.StoredValue;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexableField;
@@ -189,5 +190,10 @@ public class LazyDocument {
     public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) {
       return getRealValue().tokenStream(analyzer, reuse);
     }
+
+    @Override
+    public StoredValue storedValue() {
+      return null;
+    }
   }
 }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/asserting/AssertingStoredFieldsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/asserting/AssertingStoredFieldsFormat.java
index 56ccc9e28e0..1b2f50f6730 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/asserting/AssertingStoredFieldsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/asserting/AssertingStoredFieldsFormat.java
@@ -23,13 +23,13 @@ import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.StoredFieldsWriter;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.StoredFieldVisitor;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.tests.util.TestUtil;
 import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.BytesRef;
 
 /** Just like the default stored fields format but with additional asserts. */
 public class AssertingStoredFieldsFormat extends StoredFieldsFormat {
@@ -130,9 +130,39 @@ public class AssertingStoredFieldsFormat extends StoredFieldsFormat {
     }
 
     @Override
-    public void writeField(FieldInfo info, IndexableField field) throws IOException {
+    public void writeField(FieldInfo info, int value) throws IOException {
       assert docStatus == Status.STARTED;
-      in.writeField(info, field);
+      in.writeField(info, value);
+    }
+
+    @Override
+    public void writeField(FieldInfo info, long value) throws IOException {
+      assert docStatus == Status.STARTED;
+      in.writeField(info, value);
+    }
+
+    @Override
+    public void writeField(FieldInfo info, float value) throws IOException {
+      assert docStatus == Status.STARTED;
+      in.writeField(info, value);
+    }
+
+    @Override
+    public void writeField(FieldInfo info, double value) throws IOException {
+      assert docStatus == Status.STARTED;
+      in.writeField(info, value);
+    }
+
+    @Override
+    public void writeField(FieldInfo info, BytesRef value) throws IOException {
+      assert docStatus == Status.STARTED;
+      in.writeField(info, value);
+    }
+
+    @Override
+    public void writeField(FieldInfo info, String value) throws IOException {
+      assert docStatus == Status.STARTED;
+      in.writeField(info, value);
     }
 
     @Override
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/cranky/CrankyStoredFieldsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/cranky/CrankyStoredFieldsFormat.java
index 3637ba2f8d1..0bb54e2b899 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/cranky/CrankyStoredFieldsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/codecs/cranky/CrankyStoredFieldsFormat.java
@@ -24,12 +24,12 @@ import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.StoredFieldsWriter;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.MergeState;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.BytesRef;
 
 class CrankyStoredFieldsFormat extends StoredFieldsFormat {
   final StoredFieldsFormat delegate;
@@ -108,11 +108,51 @@ class CrankyStoredFieldsFormat extends StoredFieldsFormat {
     }
 
     @Override
-    public void writeField(FieldInfo info, IndexableField field) throws IOException {
+    public void writeField(FieldInfo info, int value) throws IOException {
       if (random.nextInt(10000) == 0) {
         throw new IOException("Fake IOException from StoredFieldsWriter.writeField()");
       }
-      delegate.writeField(info, field);
+      delegate.writeField(info, value);
+    }
+
+    @Override
+    public void writeField(FieldInfo info, long value) throws IOException {
+      if (random.nextInt(10000) == 0) {
+        throw new IOException("Fake IOException from StoredFieldsWriter.writeField()");
+      }
+      delegate.writeField(info, value);
+    }
+
+    @Override
+    public void writeField(FieldInfo info, float value) throws IOException {
+      if (random.nextInt(10000) == 0) {
+        throw new IOException("Fake IOException from StoredFieldsWriter.writeField()");
+      }
+      delegate.writeField(info, value);
+    }
+
+    @Override
+    public void writeField(FieldInfo info, double value) throws IOException {
+      if (random.nextInt(10000) == 0) {
+        throw new IOException("Fake IOException from StoredFieldsWriter.writeField()");
+      }
+      delegate.writeField(info, value);
+    }
+
+    @Override
+    public void writeField(FieldInfo info, BytesRef value) throws IOException {
+      if (random.nextInt(10000) == 0) {
+        throw new IOException("Fake IOException from StoredFieldsWriter.writeField()");
+      }
+      delegate.writeField(info, value);
+    }
+
+    @Override
+    public void writeField(FieldInfo info, String value) throws IOException {
+      if (random.nextInt(10000) == 0) {
+        throw new IOException("Fake IOException from StoredFieldsWriter.writeField()");
+      }
+      delegate.writeField(info, value);
     }
 
     @Override
diff --git a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseIndexFileFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseIndexFileFormatTestCase.java
index 3d879520f65..b1550bb727a 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseIndexFileFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/tests/index/BaseIndexFileFormatTestCase.java
@@ -50,6 +50,7 @@ import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.StoredValue;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.EmptyDocValuesProducer;
@@ -570,7 +571,29 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
     try (StoredFieldsWriter consumer =
         codec.storedFieldsFormat().fieldsWriter(dir, segmentInfo, writeState.context)) {
       consumer.startDocument();
-      consumer.writeField(field, customField);
+      StoredValue value = customField.storedValue();
+      switch (value.getType()) {
+        case INTEGER:
+          consumer.writeField(field, value.getIntValue());
+          break;
+        case LONG:
+          consumer.writeField(field, value.getLongValue());
+          break;
+        case FLOAT:
+          consumer.writeField(field, value.getFloatValue());
+          break;
+        case DOUBLE:
+          consumer.writeField(field, value.getDoubleValue());
+          break;
+        case BINARY:
+          consumer.writeField(field, value.getBinaryValue());
+          break;
+        case STRING:
+          consumer.writeField(field, value.getStringValue());
+          break;
+        default:
+          throw new AssertionError();
+      }
       consumer.finishDocument();
       consumer.finish(1);
       IOUtils.close(consumer);