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/21 13:05:00 UTC

[lucene] 02/02: Skip the TokenStream overhead when indexing simple keywords. (#12139)

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

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

commit 668246439ddfcc6ece0b1ee4a06d51e2693b6ff3
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Tue Feb 21 14:00:11 2023 +0100

    Skip the TokenStream overhead when indexing simple keywords. (#12139)
    
    Indexing simple keywords through a `TokenStream` abstraction introduces a bit
    of overhead due to attribute management. Not much, but indexing keywords boils
    down to adding to a hash map and appending to a postings list, which is quite
    cheap too so even some low overhead can significantly impact indexing speed.
---
 lucene/CHANGES.txt                                 |   2 +
 .../src/java/org/apache/lucene/document/Field.java |   5 +
 .../org/apache/lucene/document/InvertableType.java |  40 ++++
 .../org/apache/lucene/document/KeywordField.java   |  17 +-
 .../org/apache/lucene/document/StringField.java    |  15 ++
 .../org/apache/lucene/index/FieldInvertState.java  |  18 +-
 .../lucene/index/FreqProxTermsWriterPerField.java  |   4 +-
 .../org/apache/lucene/index/IndexableField.java    |   7 +
 .../org/apache/lucene/index/IndexingChain.java     |  60 ++++++
 .../lucene/index/TermVectorsConsumerPerField.java  |   3 +
 .../apache/lucene/index/TestDocumentWriter.java    | 228 ++++++++++++++++++++-
 .../lucene/index/TestExceedMaxTermLength.java      |  67 +++++-
 .../org/apache/lucene/index/TestFieldReuse.java    |  12 +-
 .../apache/lucene/index/TestIndexableField.java    |  11 +
 .../apache/lucene/index/memory/MemoryIndex.java    |  30 +++
 .../lucene/index/memory/TestMemoryIndex.java       | 149 ++++++++++++++
 .../apache/lucene/misc/document/LazyDocument.java  |   8 +-
 17 files changed, 654 insertions(+), 22 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 54b411b53e2..82cf4fa3075 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -40,6 +40,8 @@ Optimizations
 
 * GITHUB#12155: Speed up DocValuesRewriteMethod by making use of sortedness. (Greg Miller)
 
+* GITHUB#12139: Faster indexing of string fields. (Adrien Grand)
+
 Bug Fixes
 ---------------------
 (No changes)
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 db038ff37cc..68a9059039a 100644
--- a/lucene/core/src/java/org/apache/lucene/document/Field.java
+++ b/lucene/core/src/java/org/apache/lucene/document/Field.java
@@ -449,6 +449,11 @@ public class Field implements IndexableField {
     return type;
   }
 
+  @Override
+  public InvertableType invertableType() {
+    return InvertableType.TOKEN_STREAM;
+  }
+
   @Override
   public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) {
     if (fieldType().indexOptions() == IndexOptions.NONE) {
diff --git a/lucene/core/src/java/org/apache/lucene/document/InvertableType.java b/lucene/core/src/java/org/apache/lucene/document/InvertableType.java
new file mode 100644
index 00000000000..febc660817b
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/InvertableType.java
@@ -0,0 +1,40 @@
+/*
+ * 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 org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexableField;
+
+/** Describes how an {@link IndexableField} should be inverted for indexing terms and postings. */
+public enum InvertableType {
+
+  /**
+   * The field should be treated as a single value whose binary content is returned by {@link
+   * IndexableField#binaryValue()}. The term frequency is assumed to be one. If you need to index
+   * multiple values, you should pass multiple {@link IndexableField} instances to the {@link
+   * IndexWriter}. If the same value is provided multiple times, the term frequency will be equal to
+   * the number of times that this value occurred in the same document.
+   */
+  BINARY,
+
+  /**
+   * The field should be inverted through its {@link
+   * IndexableField#tokenStream(org.apache.lucene.analysis.Analyzer,
+   * org.apache.lucene.analysis.TokenStream)}.
+   */
+  TOKEN_STREAM;
+}
diff --git a/lucene/core/src/java/org/apache/lucene/document/KeywordField.java b/lucene/core/src/java/org/apache/lucene/document/KeywordField.java
index 70b27ad671a..6ff0ecaf4dd 100644
--- a/lucene/core/src/java/org/apache/lucene/document/KeywordField.java
+++ b/lucene/core/src/java/org/apache/lucene/document/KeywordField.java
@@ -63,6 +63,7 @@ public class KeywordField extends Field {
     FIELD_TYPE_STORED.freeze();
   }
 
+  private BytesRef binaryValue;
   private final StoredValue storedValue;
 
   /**
@@ -75,6 +76,7 @@ public class KeywordField extends Field {
    */
   public KeywordField(String name, BytesRef value, Store stored) {
     super(name, value, stored == Field.Store.YES ? FIELD_TYPE_STORED : FIELD_TYPE);
+    this.binaryValue = value;
     if (stored == Store.YES) {
       storedValue = new StoredValue(value);
     } else {
@@ -92,6 +94,7 @@ public class KeywordField extends Field {
    */
   public KeywordField(String name, String value, Store stored) {
     super(name, value, stored == Field.Store.YES ? FIELD_TYPE_STORED : FIELD_TYPE);
+    this.binaryValue = new BytesRef(value);
     if (stored == Store.YES) {
       storedValue = new StoredValue(value);
     } else {
@@ -101,17 +104,18 @@ public class KeywordField extends Field {
 
   @Override
   public BytesRef binaryValue() {
-    BytesRef binaryValue = super.binaryValue();
-    if (binaryValue != null) {
-      return binaryValue;
-    } else {
-      return new BytesRef(stringValue());
-    }
+    return binaryValue;
+  }
+
+  @Override
+  public InvertableType invertableType() {
+    return InvertableType.BINARY;
   }
 
   @Override
   public void setStringValue(String value) {
     super.setStringValue(value);
+    binaryValue = new BytesRef(value);
     if (storedValue != null) {
       storedValue.setStringValue(value);
     }
@@ -120,6 +124,7 @@ public class KeywordField extends Field {
   @Override
   public void setBytesValue(BytesRef value) {
     super.setBytesValue(value);
+    binaryValue = value;
     if (storedValue != null) {
       storedValue.setBinaryValue(value);
     }
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 fb242346258..4a502d79f89 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,7 @@ public final class StringField extends Field {
     TYPE_STORED.freeze();
   }
 
+  private BytesRef binaryValue;
   private final StoredValue storedValue;
 
   /**
@@ -57,6 +58,7 @@ 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);
+    binaryValue = new BytesRef(value);
     if (stored == Store.YES) {
       storedValue = new StoredValue(value);
     } else {
@@ -76,6 +78,7 @@ 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);
+    binaryValue = value;
     if (stored == Store.YES) {
       storedValue = new StoredValue(value);
     } else {
@@ -83,9 +86,20 @@ public final class StringField extends Field {
     }
   }
 
+  @Override
+  public InvertableType invertableType() {
+    return InvertableType.BINARY;
+  }
+
+  @Override
+  public BytesRef binaryValue() {
+    return binaryValue;
+  }
+
   @Override
   public void setStringValue(String value) {
     super.setStringValue(value);
+    binaryValue = new BytesRef(value);
     if (storedValue != null) {
       storedValue.setStringValue(value);
     }
@@ -94,6 +108,7 @@ public final class StringField extends Field {
   @Override
   public void setBytesValue(BytesRef value) {
     super.setBytesValue(value);
+    binaryValue = value;
     if (storedValue != null) {
       storedValue.setBinaryValue(value);
     }
diff --git a/lucene/core/src/java/org/apache/lucene/index/FieldInvertState.java b/lucene/core/src/java/org/apache/lucene/index/FieldInvertState.java
index 4bf16bb6c27..d5357fa4875 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FieldInvertState.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FieldInvertState.java
@@ -96,11 +96,19 @@ public final class FieldInvertState {
   void setAttributeSource(AttributeSource attributeSource) {
     if (this.attributeSource != attributeSource) {
       this.attributeSource = attributeSource;
-      termAttribute = attributeSource.getAttribute(TermToBytesRefAttribute.class);
-      termFreqAttribute = attributeSource.addAttribute(TermFrequencyAttribute.class);
-      posIncrAttribute = attributeSource.addAttribute(PositionIncrementAttribute.class);
-      offsetAttribute = attributeSource.addAttribute(OffsetAttribute.class);
-      payloadAttribute = attributeSource.getAttribute(PayloadAttribute.class);
+      if (attributeSource == null) {
+        termAttribute = null;
+        termFreqAttribute = null;
+        posIncrAttribute = null;
+        offsetAttribute = null;
+        payloadAttribute = null;
+      } else {
+        termAttribute = attributeSource.getAttribute(TermToBytesRefAttribute.class);
+        termFreqAttribute = attributeSource.addAttribute(TermFrequencyAttribute.class);
+        posIncrAttribute = attributeSource.addAttribute(PositionIncrementAttribute.class);
+        offsetAttribute = attributeSource.addAttribute(OffsetAttribute.class);
+        payloadAttribute = attributeSource.getAttribute(PayloadAttribute.class);
+      }
     }
   }
 
diff --git a/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java b/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
index b05747b8755..04d949f5ef9 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
@@ -144,7 +144,7 @@ final class FreqProxTermsWriterPerField extends TermsHashPerField {
 
     if (!hasFreq) {
       assert postings.termFreqs == null;
-      if (termFreqAtt.getTermFrequency() != 1) {
+      if (termFreqAtt != null && termFreqAtt.getTermFrequency() != 1) {
         throw new IllegalStateException(
             "field \""
                 + getFieldName()
@@ -203,7 +203,7 @@ final class FreqProxTermsWriterPerField extends TermsHashPerField {
   }
 
   private int getTermFreq() {
-    int freq = termFreqAtt.getTermFrequency();
+    int freq = termFreqAtt == null ? 1 : termFreqAtt.getTermFrequency();
     if (freq != 1) {
       if (hasProx) {
         throw new IllegalStateException(
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 0715d327b83..7293580df82 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.InvertableType;
 import org.apache.lucene.document.StoredValue;
 import org.apache.lucene.util.BytesRef;
 
@@ -75,4 +76,10 @@ public interface IndexableField {
    * if the field stored.
    */
   public StoredValue storedValue();
+
+  /**
+   * Describes how this field should be inverted. This must return a non-null value if the field
+   * indexes terms and postings.
+   */
+  public InvertableType invertableType();
 }
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 3f4930403b6..a68a84ff5ac 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexingChain.java
@@ -1111,11 +1111,27 @@ final class IndexingChain implements Accountable {
      * this field name in this document.
      */
     public void invert(int docID, IndexableField field, boolean first) throws IOException {
+      assert field.fieldType().indexOptions().compareTo(IndexOptions.DOCS) >= 0;
+
       if (first) {
         // First time we're seeing this field (indexed) in this document
         invertState.reset();
       }
 
+      switch (field.invertableType()) {
+        case BINARY:
+          invertTerm(docID, field, first);
+          break;
+        case TOKEN_STREAM:
+          invertTokenStream(docID, field, first);
+          break;
+        default:
+          throw new AssertionError();
+      }
+    }
+
+    private void invertTokenStream(int docID, IndexableField field, boolean first)
+        throws IOException {
       final boolean analyzed = field.fieldType().tokenized() && analyzer != null;
       /*
        * To assist people in tracking down problems in analysis components, we wish to write the field name to the infostream
@@ -1258,6 +1274,50 @@ final class IndexingChain implements Accountable {
         invertState.offset += analyzer.getOffsetGap(fieldInfo.name);
       }
     }
+
+    private void invertTerm(int docID, IndexableField field, boolean first) throws IOException {
+      BytesRef binaryValue = field.binaryValue();
+      if (binaryValue == null) {
+        throw new IllegalArgumentException(
+            "Field "
+                + field.name()
+                + " returns TERM for invertableType() and null for binaryValue(), which is illegal");
+      }
+      final IndexableFieldType fieldType = field.fieldType();
+      if (fieldType.tokenized()
+          || fieldType.indexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) > 0
+          || fieldType.storeTermVectorPositions()
+          || fieldType.storeTermVectorOffsets()
+          || fieldType.storeTermVectorPayloads()) {
+        throw new IllegalArgumentException(
+            "Fields that are tokenized or index proximity data must produce a non-null TokenStream, but "
+                + field.name()
+                + " did not");
+      }
+      invertState.setAttributeSource(null);
+      invertState.position++;
+      invertState.length++;
+      termsHashPerField.start(field, first);
+      invertState.length = Math.addExact(invertState.length, 1);
+      try {
+        termsHashPerField.add(binaryValue, docID);
+      } catch (MaxBytesLengthExceededException e) {
+        byte[] prefix = new byte[30];
+        System.arraycopy(binaryValue.bytes, binaryValue.offset, prefix, 0, 30);
+        String msg =
+            "Document contains at least one immense term in field=\""
+                + fieldInfo.name
+                + "\" (whose length is longer than the max length "
+                + IndexWriter.MAX_TERM_LENGTH
+                + "), all of which were skipped. The prefix of the first immense term is: '"
+                + Arrays.toString(prefix)
+                + "...'";
+        if (infoStream.isEnabled("IW")) {
+          infoStream.message("IW", "ERROR: " + msg);
+        }
+        throw new IllegalArgumentException(msg, e);
+      }
+    }
   }
 
   DocIdSetIterator getHasDocValues(String field) {
diff --git a/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java b/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java
index a4c24482201..f2e350de78d 100644
--- a/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java
+++ b/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java
@@ -284,6 +284,9 @@ final class TermVectorsConsumerPerField extends TermsHashPerField {
   }
 
   private int getTermFreq() {
+    if (termFreqAtt == null) {
+      return 1;
+    }
     int freq = termFreqAtt.getTermFrequency();
     if (freq != 1) {
       if (doVectorPositions) {
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java b/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java
index 1ba00844885..7d8cdf95d23 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDocumentWriter.java
@@ -17,6 +17,9 @@
 package org.apache.lucene.index;
 
 import java.io.IOException;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.function.Function;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenFilter;
@@ -31,14 +34,17 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.document.InvertableType;
 import org.apache.lucene.document.KnnFloatVectorField;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.SortedNumericDocValuesField;
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.document.StoredField;
+import org.apache.lucene.document.StoredValue;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.tests.analysis.MockAnalyzer;
@@ -67,10 +73,6 @@ public class TestDocumentWriter extends LuceneTestCase {
     super.tearDown();
   }
 
-  public void test() {
-    assertTrue(dir != null);
-  }
-
   public void testAddDocument() throws Exception {
     Document testDoc = new Document();
     DocHelper.setupDoc(testDoc);
@@ -385,4 +387,222 @@ public class TestDocumentWriter extends LuceneTestCase {
             new KnnFloatVectorField(
                 field, new float[] {1, 2, 3, 4}, VectorSimilarityFunction.EUCLIDEAN));
   }
+
+  private static class MockIndexableField implements IndexableField {
+
+    private final String field;
+    private final BytesRef value;
+    private final IndexableFieldType fieldType;
+
+    MockIndexableField(String field, BytesRef value, IndexableFieldType fieldType) {
+      this.field = field;
+      this.value = value;
+      this.fieldType = fieldType;
+    }
+
+    @Override
+    public String name() {
+      return field;
+    }
+
+    @Override
+    public IndexableFieldType fieldType() {
+      return fieldType;
+    }
+
+    @Override
+    public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) {
+      return null;
+    }
+
+    @Override
+    public BytesRef binaryValue() {
+      return value;
+    }
+
+    @Override
+    public String stringValue() {
+      return null;
+    }
+
+    @Override
+    public Reader readerValue() {
+      return null;
+    }
+
+    @Override
+    public Number numericValue() {
+      return null;
+    }
+
+    @Override
+    public StoredValue storedValue() {
+      return null;
+    }
+
+    @Override
+    public InvertableType invertableType() {
+      return InvertableType.BINARY;
+    }
+  }
+
+  public void testIndexBinaryValueWithoutTokenStream() throws IOException {
+    List<FieldType> illegalFieldTypes = new ArrayList<>();
+    {
+      FieldType illegalFT = new FieldType();
+      // cannot index a tokenized binary field
+      illegalFT.setTokenized(true);
+      illegalFT.setIndexOptions(IndexOptions.DOCS);
+      illegalFT.freeze();
+      illegalFieldTypes.add(illegalFT);
+    }
+    {
+      FieldType illegalFT = new FieldType();
+      illegalFT.setTokenized(false);
+      // cannot index positions on a binary field
+      illegalFT.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+      illegalFT.freeze();
+      illegalFieldTypes.add(illegalFT);
+    }
+    {
+      FieldType illegalFT = new FieldType();
+      illegalFT.setTokenized(false);
+      illegalFT.setIndexOptions(IndexOptions.DOCS);
+      illegalFT.setStoreTermVectors(true);
+      // cannot index term vector positions
+      illegalFT.setStoreTermVectorPositions(true);
+      illegalFT.freeze();
+      illegalFieldTypes.add(illegalFT);
+    }
+    {
+      FieldType illegalFT = new FieldType();
+      illegalFT.setTokenized(false);
+      illegalFT.setIndexOptions(IndexOptions.DOCS);
+      illegalFT.setStoreTermVectors(true);
+      // cannot index term vector offsets
+      illegalFT.setStoreTermVectorOffsets(true);
+      illegalFT.freeze();
+      illegalFieldTypes.add(illegalFT);
+    }
+
+    for (FieldType ft : illegalFieldTypes) {
+      try (IndexWriter w =
+          new IndexWriter(dir, newIndexWriterConfig().setOpenMode(OpenMode.CREATE))) {
+        MockIndexableField field = new MockIndexableField("field", new BytesRef("a"), ft);
+        Document doc = new Document();
+        doc.add(field);
+        expectThrows(IllegalArgumentException.class, () -> w.addDocument(doc));
+      }
+    }
+
+    try (IndexWriter w =
+        new IndexWriter(dir, newIndexWriterConfig().setOpenMode(OpenMode.CREATE))) {
+      // Field that has both a null token stream and a null binary value
+      MockIndexableField field = new MockIndexableField("field", null, StringField.TYPE_NOT_STORED);
+      Document doc = new Document();
+      doc.add(field);
+      expectThrows(IllegalArgumentException.class, () -> w.addDocument(doc));
+    }
+
+    List<FieldType> legalFieldTypes = new ArrayList<>();
+    {
+      FieldType ft = new FieldType();
+      ft.setTokenized(false);
+      ft.setIndexOptions(IndexOptions.DOCS);
+      ft.setOmitNorms(false);
+      ft.freeze();
+      legalFieldTypes.add(ft);
+    }
+    {
+      FieldType ft = new FieldType();
+      ft.setTokenized(false);
+      ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
+      ft.setOmitNorms(false);
+      ft.freeze();
+      legalFieldTypes.add(ft);
+    }
+    {
+      FieldType ft = new FieldType();
+      ft.setTokenized(false);
+      ft.setIndexOptions(IndexOptions.DOCS);
+      ft.setOmitNorms(true);
+      ft.freeze();
+      legalFieldTypes.add(ft);
+    }
+    {
+      FieldType ft = new FieldType();
+      ft.setTokenized(false);
+      ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
+      ft.setOmitNorms(true);
+      ft.freeze();
+      legalFieldTypes.add(ft);
+    }
+    {
+      FieldType ft = new FieldType();
+      ft.setTokenized(false);
+      ft.setIndexOptions(IndexOptions.DOCS);
+      ft.setStoreTermVectors(true);
+      ft.freeze();
+      legalFieldTypes.add(ft);
+    }
+    {
+      FieldType ft = new FieldType();
+      ft.setTokenized(false);
+      ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
+      ft.setStoreTermVectors(true);
+      ft.freeze();
+      legalFieldTypes.add(ft);
+    }
+
+    for (FieldType ft : legalFieldTypes) {
+      try (IndexWriter w =
+          new IndexWriter(dir, newIndexWriterConfig().setOpenMode(OpenMode.CREATE))) {
+        MockIndexableField field = new MockIndexableField("field", new BytesRef("a"), ft);
+        Document doc = new Document();
+        doc.add(field);
+        doc.add(field);
+        w.addDocument(doc);
+      }
+
+      try (DirectoryReader reader = DirectoryReader.open(dir)) {
+        LeafReader leafReader = getOnlyLeafReader(reader);
+
+        {
+          Terms terms = leafReader.terms("field");
+          assertEquals(1, terms.getSumDocFreq());
+          if (ft.indexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0) {
+            assertEquals(2, terms.getSumTotalTermFreq());
+          } else {
+            assertEquals(1, terms.getSumTotalTermFreq());
+          }
+          TermsEnum termsEnum = terms.iterator();
+          assertTrue(termsEnum.seekExact(new BytesRef("a")));
+          PostingsEnum pe = termsEnum.postings(null, PostingsEnum.ALL);
+          assertEquals(0, pe.nextDoc());
+          if (ft.indexOptions().compareTo(IndexOptions.DOCS_AND_FREQS) >= 0) {
+            assertEquals(2, pe.freq());
+          } else {
+            assertEquals(1, pe.freq());
+          }
+          assertEquals(-1, pe.nextPosition());
+          assertEquals(DocIdSetIterator.NO_MORE_DOCS, pe.nextDoc());
+        }
+
+        if (ft.storeTermVectors()) {
+          Terms tvTerms = leafReader.termVectors().get(0).terms("field");
+          assertEquals(1, tvTerms.getSumDocFreq());
+          assertEquals(2, tvTerms.getSumTotalTermFreq());
+          TermsEnum tvTermsEnum = tvTerms.iterator();
+          assertTrue(tvTermsEnum.seekExact(new BytesRef("a")));
+          PostingsEnum pe = tvTermsEnum.postings(null, PostingsEnum.ALL);
+          assertEquals(0, pe.nextDoc());
+          assertEquals(2, pe.freq());
+          assertEquals(-1, pe.nextPosition());
+          assertEquals(DocIdSetIterator.NO_MORE_DOCS, pe.nextDoc());
+        } else {
+          assertNull(leafReader.termVectors().get(0));
+        }
+      }
+    }
+  }
 }
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestExceedMaxTermLength.java b/lucene/core/src/test/org/apache/lucene/index/TestExceedMaxTermLength.java
index 11f58b88348..3b252f5d5de 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestExceedMaxTermLength.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestExceedMaxTermLength.java
@@ -16,7 +16,9 @@
  */
 package org.apache.lucene.index;
 
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 import java.io.IOException;
+import java.util.Arrays;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
@@ -24,6 +26,7 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.tests.analysis.MockAnalyzer;
 import org.apache.lucene.tests.util.LuceneTestCase;
 import org.apache.lucene.tests.util.TestUtil;
+import org.apache.lucene.util.BytesRef;
 import org.junit.After;
 import org.junit.Before;
 
@@ -50,7 +53,7 @@ public class TestExceedMaxTermLength extends LuceneTestCase {
     dir = null;
   }
 
-  public void test() throws Exception {
+  public void testTokenStream() throws Exception {
 
     MockAnalyzer mockAnalyzer = new MockAnalyzer(random());
     mockAnalyzer.setMaxTokenLength(Integer.MAX_VALUE);
@@ -109,4 +112,66 @@ public class TestExceedMaxTermLength extends LuceneTestCase {
       w.close();
     }
   }
+
+  public void testBinaryValue() throws Exception {
+
+    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
+    try {
+      final FieldType ft = new FieldType();
+      ft.setIndexOptions(
+          RandomPicks.randomFrom(
+              random(), Arrays.asList(IndexOptions.DOCS, IndexOptions.DOCS_AND_FREQS)));
+      ft.setStored(random().nextBoolean());
+      ft.setTokenized(false);
+      ft.freeze();
+
+      final Document doc = new Document();
+      if (random().nextBoolean()) {
+        // totally ok short field value
+        doc.add(
+            new Field(
+                TestUtil.randomSimpleString(random(), 1, 10),
+                TestUtil.randomBinaryTerm(random(), 10),
+                ft));
+      }
+      // problematic field
+      final String name = TestUtil.randomSimpleString(random(), 1, 50);
+      final BytesRef value =
+          TestUtil.randomBinaryTerm(
+              random(), TestUtil.nextInt(random(), minTestTermLength, maxTestTermLength));
+      final Field f = new Field(name, value, ft);
+      if (random().nextBoolean()) {
+        // totally ok short field value
+        doc.add(
+            new Field(
+                TestUtil.randomSimpleString(random(), 1, 10),
+                TestUtil.randomBinaryTerm(random(), 10),
+                ft));
+      }
+      doc.add(f);
+
+      IllegalArgumentException expected =
+          expectThrows(
+              IllegalArgumentException.class,
+              () -> {
+                w.addDocument(doc);
+              });
+      String maxLengthMsg = String.valueOf(IndexWriter.MAX_TERM_LENGTH);
+      String msg = expected.getMessage();
+      assertTrue(
+          "IllegalArgumentException didn't mention 'immense term': " + msg,
+          msg.contains("immense term"));
+      assertTrue(
+          "IllegalArgumentException didn't mention max length (" + maxLengthMsg + "): " + msg,
+          msg.contains(maxLengthMsg));
+      assertTrue(
+          "IllegalArgumentException didn't mention field name (" + name + "): " + msg,
+          msg.contains(name));
+      assertTrue(
+          "IllegalArgumentException didn't mention original message: " + msg,
+          msg.contains("bytes can be at most") && msg.contains("in length; got"));
+    } finally {
+      w.close();
+    }
+  }
 }
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 1a81651800b..0dabb5b17f8 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.InvertableType;
 import org.apache.lucene.document.StoredValue;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.store.Directory;
@@ -34,20 +35,20 @@ import org.apache.lucene.util.BytesRef;
 public class TestFieldReuse extends BaseTokenStreamTestCase {
 
   public void testStringField() throws IOException {
-    StringField stringField = new StringField("foo", "bar", Field.Store.NO);
+    Field stringField = new Field("foo", "bar", StringField.TYPE_NOT_STORED);
 
     // passing null
     TokenStream ts = stringField.tokenStream(null, null);
     assertTokenStreamContents(ts, new String[] {"bar"}, new int[] {0}, new int[] {3});
 
     // now reuse previous stream
-    stringField = new StringField("foo", "baz", Field.Store.NO);
+    stringField = new Field("foo", "baz", StringField.TYPE_NOT_STORED);
     TokenStream ts2 = stringField.tokenStream(null, ts);
     assertSame(ts, ts2);
     assertTokenStreamContents(ts, new String[] {"baz"}, new int[] {0}, new int[] {3});
 
     // pass a bogus stream and ensure it's still ok
-    stringField = new StringField("foo", "beer", Field.Store.NO);
+    stringField = new Field("foo", "beer", StringField.TYPE_NOT_STORED);
     TokenStream bogus = new CannedTokenStream();
     ts = stringField.tokenStream(null, bogus);
     assertNotSame(ts, bogus);
@@ -98,6 +99,11 @@ public class TestFieldReuse extends BaseTokenStreamTestCase {
     public StoredValue storedValue() {
       return null;
     }
+
+    @Override
+    public InvertableType invertableType() {
+      return InvertableType.TOKEN_STREAM;
+    }
   }
 
   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 d5f52338519..886417fabc8 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexableField.java
@@ -26,6 +26,7 @@ import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.InvertableType;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.document.StoredValue;
 import org.apache.lucene.search.BooleanClause;
@@ -198,6 +199,11 @@ public class TestIndexableField extends LuceneTestCase {
         return null;
       }
     }
+
+    @Override
+    public InvertableType invertableType() {
+      return InvertableType.TOKEN_STREAM;
+    }
   }
 
   // Silly test showing how to index documents w/o using Lucene's core
@@ -405,6 +411,11 @@ public class TestIndexableField extends LuceneTestCase {
     public StoredValue storedValue() {
       return null;
     }
+
+    @Override
+    public InvertableType invertableType() {
+      return InvertableType.TOKEN_STREAM;
+    }
   }
 
   // LUCENE-5611
diff --git a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
index ab59337bd92..82fb61b39ba 100644
--- a/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
+++ b/lucene/memory/src/java/org/apache/lucene/index/memory/MemoryIndex.java
@@ -391,6 +391,13 @@ public class MemoryIndex {
     }
     if (tokenStream != null) {
       storeTerms(info, tokenStream, positionIncrementGap, offsetGap);
+    } else if (field.fieldType().indexOptions().compareTo(IndexOptions.DOCS) >= 0) {
+      BytesRef binaryValue = field.binaryValue();
+      if (binaryValue == null) {
+        throw new IllegalArgumentException(
+            "Indexed field must provide a TokenStream or a binary value");
+      }
+      storeTerm(info, binaryValue);
     }
 
     DocValuesType docValuesType = field.fieldType().docValuesType();
@@ -632,6 +639,29 @@ public class MemoryIndex {
     }
   }
 
+  private void storeTerm(Info info, BytesRef term) {
+    info.numTokens++;
+    int ord = info.terms.add(term);
+    if (ord < 0) {
+      ord = -ord - 1;
+      postingsWriter.reset(info.sliceArray.end[ord]);
+    } else {
+      info.sliceArray.start[ord] = postingsWriter.startNewSlice();
+    }
+    info.sliceArray.freq[ord]++;
+    info.maxTermFrequency = Math.max(info.maxTermFrequency, info.sliceArray.freq[ord]);
+    info.sumTotalTermFreq++;
+    postingsWriter.writeInt(info.lastPosition++); // fake position
+    if (storeOffsets) { // fake offsests
+      postingsWriter.writeInt(0);
+      postingsWriter.writeInt(0);
+    }
+    if (storePayloads) {
+      postingsWriter.writeInt(-1); // fake payload
+    }
+    info.sliceArray.end[ord] = postingsWriter.getCurrentOffset();
+  }
+
   private void storeTerms(
       Info info, TokenStream tokenStream, int positionIncrementGap, int offsetGap) {
 
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 9e716484b47..7cddbe2fe3d 100644
--- a/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
+++ b/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndex.java
@@ -21,6 +21,7 @@ import static org.hamcrest.CoreMatchers.not;
 import static org.hamcrest.core.StringContains.containsString;
 
 import java.io.IOException;
+import java.io.Reader;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -31,6 +32,7 @@ import java.util.function.BiFunction;
 import java.util.function.Function;
 import java.util.stream.LongStream;
 import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.document.BinaryDocValuesField;
 import org.apache.lucene.document.BinaryPoint;
@@ -42,12 +44,14 @@ import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.FloatPoint;
 import org.apache.lucene.document.IntField;
 import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.document.InvertableType;
 import org.apache.lucene.document.LongPoint;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.SortedNumericDocValuesField;
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.document.StoredField;
+import org.apache.lucene.document.StoredValue;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.BinaryDocValues;
@@ -56,6 +60,7 @@ import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.IndexableFieldType;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.PostingsEnum;
@@ -63,6 +68,7 @@ import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.CollectionStatistics;
 import org.apache.lucene.search.DocIdSetIterator;
@@ -843,4 +849,147 @@ public class TestMemoryIndex extends LuceneTestCase {
     assertEquals(1, sndv.docValueCount());
     assertEquals(50, sndv.nextValue());
   }
+
+  private static class MockIndexableField implements IndexableField {
+
+    private final String field;
+    private final BytesRef value;
+    private final IndexableFieldType fieldType;
+
+    MockIndexableField(String field, BytesRef value, IndexableFieldType fieldType) {
+      this.field = field;
+      this.value = value;
+      this.fieldType = fieldType;
+    }
+
+    @Override
+    public String name() {
+      return field;
+    }
+
+    @Override
+    public IndexableFieldType fieldType() {
+      return fieldType;
+    }
+
+    @Override
+    public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) {
+      return null;
+    }
+
+    @Override
+    public BytesRef binaryValue() {
+      return value;
+    }
+
+    @Override
+    public String stringValue() {
+      return null;
+    }
+
+    @Override
+    public Reader readerValue() {
+      return null;
+    }
+
+    @Override
+    public Number numericValue() {
+      return null;
+    }
+
+    @Override
+    public StoredValue storedValue() {
+      return null;
+    }
+
+    @Override
+    public InvertableType invertableType() {
+      return InvertableType.BINARY;
+    }
+  }
+
+  public void testKeywordWithoutTokenStream() throws IOException {
+    List<FieldType> legalFieldTypes = new ArrayList<>();
+    {
+      FieldType ft = new FieldType();
+      ft.setTokenized(false);
+      ft.setIndexOptions(IndexOptions.DOCS);
+      ft.setOmitNorms(false);
+      ft.freeze();
+      legalFieldTypes.add(ft);
+    }
+    {
+      FieldType ft = new FieldType();
+      ft.setTokenized(false);
+      ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
+      ft.setOmitNorms(false);
+      ft.freeze();
+      legalFieldTypes.add(ft);
+    }
+    {
+      FieldType ft = new FieldType();
+      ft.setTokenized(false);
+      ft.setIndexOptions(IndexOptions.DOCS);
+      ft.setOmitNorms(true);
+      ft.freeze();
+      legalFieldTypes.add(ft);
+    }
+    {
+      FieldType ft = new FieldType();
+      ft.setTokenized(false);
+      ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
+      ft.setOmitNorms(true);
+      ft.freeze();
+      legalFieldTypes.add(ft);
+    }
+    {
+      FieldType ft = new FieldType();
+      ft.setTokenized(false);
+      ft.setIndexOptions(IndexOptions.DOCS);
+      ft.setStoreTermVectors(true);
+      ft.freeze();
+      legalFieldTypes.add(ft);
+    }
+    {
+      FieldType ft = new FieldType();
+      ft.setTokenized(false);
+      ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
+      ft.setStoreTermVectors(true);
+      ft.freeze();
+      legalFieldTypes.add(ft);
+    }
+
+    for (FieldType ft : legalFieldTypes) {
+      MockIndexableField field = new MockIndexableField("field", new BytesRef("a"), ft);
+      MemoryIndex index = MemoryIndex.fromDocument(Arrays.asList(field, field), null);
+      LeafReader leafReader = index.createSearcher().getIndexReader().leaves().get(0).reader();
+      {
+        Terms terms = leafReader.terms("field");
+        assertEquals(1, terms.getSumDocFreq());
+        assertEquals(2, terms.getSumTotalTermFreq());
+        TermsEnum termsEnum = terms.iterator();
+        assertTrue(termsEnum.seekExact(new BytesRef("a")));
+        PostingsEnum pe = termsEnum.postings(null, PostingsEnum.ALL);
+        assertEquals(0, pe.nextDoc());
+        assertEquals(2, pe.freq());
+        assertEquals(0, pe.nextPosition());
+        assertEquals(1, pe.nextPosition());
+        assertEquals(DocIdSetIterator.NO_MORE_DOCS, pe.nextDoc());
+      }
+
+      if (ft.storeTermVectors()) {
+        Terms tvTerms = leafReader.termVectors().get(0).terms("field");
+        assertEquals(1, tvTerms.getSumDocFreq());
+        assertEquals(2, tvTerms.getSumTotalTermFreq());
+        TermsEnum tvTermsEnum = tvTerms.iterator();
+        assertTrue(tvTermsEnum.seekExact(new BytesRef("a")));
+        PostingsEnum pe = tvTermsEnum.postings(null, PostingsEnum.ALL);
+        assertEquals(0, pe.nextDoc());
+        assertEquals(2, pe.freq());
+        assertEquals(0, pe.nextPosition());
+        assertEquals(1, pe.nextPosition());
+        assertEquals(DocIdSetIterator.NO_MORE_DOCS, pe.nextDoc());
+      }
+    }
+  }
 }
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 581eb90785c..3962f0ee2b6 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.InvertableType;
 import org.apache.lucene.document.StoredValue;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexReader;
@@ -193,7 +194,12 @@ public class LazyDocument {
 
     @Override
     public StoredValue storedValue() {
-      return null;
+      return getRealValue().storedValue();
+    }
+
+    @Override
+    public InvertableType invertableType() {
+      return getRealValue().invertableType();
     }
   }
 }