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/07 17:19:16 UTC

[lucene] branch main updated: Introduce a new `KeywordField`. (#12054)

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 ab074d54839 Introduce a new `KeywordField`. (#12054)
ab074d54839 is described below

commit ab074d54839e236fe8976971adf10f509957a342
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Tue Feb 7 18:19:09 2023 +0100

    Introduce a new `KeywordField`. (#12054)
    
    `KeywordField` is a combination of `StringField` and `SortedSetDocValuesField`,
    similarly to how `LongField` is a combination of `LongPoint` and
    `SortedNumericDocValuesField`. This makes it easier for users to create fields
    that can be used for filtering, sorting and faceting.
---
 lucene/CHANGES.txt                                 |   4 +-
 .../org/apache/lucene/document/KeywordField.java   | 188 +++++++++++++++++++++
 .../apache/lucene/document/TestKeywordField.java   | 125 ++++++++++++++
 .../apache/lucene/search/TestSortOptimization.java |  44 ++---
 .../lucene/search/TestSortedSetSortField.java      |  30 ++--
 .../java/org/apache/lucene/demo/IndexFiles.java    |   5 +-
 6 files changed, 355 insertions(+), 41 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index da89bc1789b..1bc6cfe3ccd 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -114,7 +114,9 @@ API Changes
 
 New Features
 ---------------------
-(No changes)
+
+* GITHUB#12054: Introduce a new KeywordField for simple and efficient
+  filtering, sorting and faceting. (Adrien Grand)
 
 Improvements
 ---------------------
diff --git a/lucene/core/src/java/org/apache/lucene/document/KeywordField.java b/lucene/core/src/java/org/apache/lucene/document/KeywordField.java
new file mode 100644
index 00000000000..70b27ad671a
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/document/KeywordField.java
@@ -0,0 +1,188 @@
+/*
+ * 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.DocValuesType;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.IndexOrDocValuesQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.SortedSetSelector;
+import org.apache.lucene.search.SortedSetSortField;
+import org.apache.lucene.search.TermInSetQuery;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Field that indexes a per-document String or {@link BytesRef} into an inverted index for fast
+ * filtering, stores values in a columnar fashion using {@link DocValuesType#SORTED_SET} doc values
+ * for sorting and faceting, and optionally stores values as stored fields for top-hits retrieval.
+ * This field does not support scoring: queries produce constant scores. If you need more
+ * fine-grained control you can use {@link StringField}, {@link SortedDocValuesField} or {@link
+ * SortedSetDocValuesField}, and {@link StoredField}.
+ *
+ * <p>This field defines static factory methods for creating common query objects:
+ *
+ * <ul>
+ *   <li>{@link #newExactQuery} for matching a value.
+ *   <li>{@link #newSetQuery} for matching any of the values coming from a set.
+ *   <li>{@link #newSortField} for matching a value.
+ * </ul>
+ */
+public class KeywordField extends Field {
+
+  private static final FieldType FIELD_TYPE = new FieldType();
+  private static final FieldType FIELD_TYPE_STORED;
+
+  static {
+    FIELD_TYPE.setIndexOptions(IndexOptions.DOCS);
+    FIELD_TYPE.setOmitNorms(true);
+    FIELD_TYPE.setTokenized(false);
+    FIELD_TYPE.setDocValuesType(DocValuesType.SORTED_SET);
+    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 KeywordField.
+   *
+   * @param name field name
+   * @param value the BytesRef value
+   * @param stored whether to store the field
+   * @throws IllegalArgumentException if the field name or value is null.
+   */
+  public KeywordField(String name, BytesRef value, Store stored) {
+    super(name, value, stored == Field.Store.YES ? FIELD_TYPE_STORED : FIELD_TYPE);
+    if (stored == Store.YES) {
+      storedValue = new StoredValue(value);
+    } else {
+      storedValue = null;
+    }
+  }
+
+  /**
+   * Creates a new KeywordField from a String value, by indexing its UTF-8 representation.
+   *
+   * @param name field name
+   * @param value the BytesRef value
+   * @param stored whether to store the field
+   * @throws IllegalArgumentException if the field name or value is null.
+   */
+  public KeywordField(String name, String value, Store stored) {
+    super(name, value, stored == Field.Store.YES ? FIELD_TYPE_STORED : FIELD_TYPE);
+    if (stored == Store.YES) {
+      storedValue = new StoredValue(value);
+    } else {
+      storedValue = null;
+    }
+  }
+
+  @Override
+  public BytesRef binaryValue() {
+    BytesRef binaryValue = super.binaryValue();
+    if (binaryValue != null) {
+      return binaryValue;
+    } else {
+      return new BytesRef(stringValue());
+    }
+  }
+
+  @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;
+  }
+
+  /**
+   * Create a query for matching an exact {@link BytesRef} value.
+   *
+   * @param field field name. must not be {@code null}.
+   * @param value exact value
+   * @throws NullPointerException if {@code field} is null.
+   * @return a query matching documents with this exact value
+   */
+  public static Query newExactQuery(String field, BytesRef value) {
+    Objects.requireNonNull(field, "field must not be null");
+    Objects.requireNonNull(value, "value must not be null");
+    return new ConstantScoreQuery(new TermQuery(new Term(field, value)));
+  }
+
+  /**
+   * Create a query for matching an exact {@link String} value.
+   *
+   * @param field field name. must not be {@code null}.
+   * @param value exact value
+   * @throws NullPointerException if {@code field} is null.
+   * @return a query matching documents with this exact value
+   */
+  public static Query newExactQuery(String field, String value) {
+    Objects.requireNonNull(value, "value must not be null");
+    return newExactQuery(field, new BytesRef(value));
+  }
+
+  /**
+   * Create a query for matching any of a set of provided {@link BytesRef} values.
+   *
+   * @param field field name. must not be {@code null}.
+   * @param values the set of values to match
+   * @throws NullPointerException if {@code field} is null.
+   * @return a query matching documents with this exact value
+   */
+  public static Query newSetQuery(String field, BytesRef... values) {
+    Objects.requireNonNull(field, "field must not be null");
+    Objects.requireNonNull(values, "values must not be null");
+    return new IndexOrDocValuesQuery(
+        new TermInSetQuery(field, values), new SortedSetDocValuesSetQuery(field, values));
+  }
+
+  /**
+   * Create a new {@link SortField} for {@link BytesRef} values.
+   *
+   * @param field field name. must not be {@code null}.
+   * @param reverse true if natural order should be reversed.
+   * @param selector custom selector type for choosing the sort value from the set.
+   */
+  public static SortField newSortField(
+      String field, boolean reverse, SortedSetSelector.Type selector) {
+    Objects.requireNonNull(field, "field must not be null");
+    Objects.requireNonNull(selector, "selector must not be null");
+    return new SortedSetSortField(field, reverse, selector);
+  }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/document/TestKeywordField.java b/lucene/core/src/test/org/apache/lucene/document/TestKeywordField.java
new file mode 100644
index 00000000000..6593a4509c6
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/document/TestKeywordField.java
@@ -0,0 +1,125 @@
+/*
+ * 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.io.IOException;
+import java.util.Collections;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.tests.util.LuceneTestCase;
+import org.apache.lucene.util.BytesRef;
+
+public class TestKeywordField extends LuceneTestCase {
+
+  public void testSetBytesValue() {
+    Field[] fields =
+        new Field[] {
+          new KeywordField("name", newBytesRef("value"), Field.Store.NO),
+          new KeywordField("name", newBytesRef("value"), Field.Store.YES)
+        };
+    for (Field field : fields) {
+      assertEquals(newBytesRef("value"), field.binaryValue());
+      assertNull(field.stringValue());
+      if (field.fieldType().stored()) {
+        assertEquals(newBytesRef("value"), field.storedValue().getBinaryValue());
+      } else {
+        assertNull(field.storedValue());
+      }
+      field.setBytesValue(newBytesRef("value2"));
+      assertEquals(newBytesRef("value2"), field.binaryValue());
+      assertNull(field.stringValue());
+      if (field.fieldType().stored()) {
+        assertEquals(newBytesRef("value2"), field.storedValue().getBinaryValue());
+      } else {
+        assertNull(field.storedValue());
+      }
+    }
+  }
+
+  public void testSetStringValue() {
+    Field[] fields =
+        new Field[] {
+          new KeywordField("name", "value", Field.Store.NO),
+          new KeywordField("name", "value", Field.Store.YES)
+        };
+    for (Field field : fields) {
+      assertEquals("value", field.stringValue());
+      assertEquals(newBytesRef("value"), field.binaryValue());
+      if (field.fieldType().stored()) {
+        assertEquals("value", field.storedValue().getStringValue());
+      } else {
+        assertNull(field.storedValue());
+      }
+      field.setStringValue("value2");
+      assertEquals("value2", field.stringValue());
+      assertEquals(newBytesRef("value2"), field.binaryValue());
+      if (field.fieldType().stored()) {
+        assertEquals("value2", field.storedValue().getStringValue());
+      } else {
+        assertNull(field.storedValue());
+      }
+    }
+  }
+
+  public void testIndexBytesValue() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
+    w.addDocument(
+        Collections.singleton(new KeywordField("field", newBytesRef("value"), Field.Store.YES)));
+    IndexReader reader = DirectoryReader.open(w);
+    w.close();
+    LeafReader leaf = getOnlyLeafReader(reader);
+    TermsEnum terms = leaf.terms("field").iterator();
+    assertEquals(new BytesRef("value"), terms.next());
+    assertNull(terms.next());
+    SortedSetDocValues values = leaf.getSortedSetDocValues("field");
+    assertTrue(values.advanceExact(0));
+    assertEquals(1, values.docValueCount());
+    assertEquals(0L, values.nextOrd());
+    assertEquals(new BytesRef("value"), values.lookupOrd(0));
+    Document storedDoc = leaf.storedFields().document(0);
+    assertEquals(new BytesRef("value"), storedDoc.getBinaryValue("field"));
+    reader.close();
+    dir.close();
+  }
+
+  public void testIndexStringValue() throws IOException {
+    Directory dir = newDirectory();
+    IndexWriter w = new IndexWriter(dir, newIndexWriterConfig());
+    w.addDocument(Collections.singleton(new KeywordField("field", "value", Field.Store.YES)));
+    IndexReader reader = DirectoryReader.open(w);
+    w.close();
+    LeafReader leaf = getOnlyLeafReader(reader);
+    TermsEnum terms = leaf.terms("field").iterator();
+    assertEquals(new BytesRef("value"), terms.next());
+    assertNull(terms.next());
+    SortedSetDocValues values = leaf.getSortedSetDocValues("field");
+    assertTrue(values.advanceExact(0));
+    assertEquals(1, values.docValueCount());
+    assertEquals(0L, values.nextOrd());
+    assertEquals(new BytesRef("value"), values.lookupOrd(0));
+    Document storedDoc = leaf.storedFields().document(0);
+    assertEquals("value", storedDoc.get("field"));
+    reader.close();
+    dir.close();
+  }
+}
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 5c0aad74d97..d30146f39a3 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestSortOptimization.java
@@ -26,15 +26,14 @@ import java.util.Collections;
 import java.util.List;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
-import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.FloatDocValuesField;
 import org.apache.lucene.document.FloatPoint;
 import org.apache.lucene.document.IntPoint;
 import org.apache.lucene.document.IntRange;
+import org.apache.lucene.document.KeywordField;
 import org.apache.lucene.document.LongField;
 import org.apache.lucene.document.LongPoint;
 import org.apache.lucene.document.NumericDocValuesField;
-import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.DirectoryReader;
@@ -809,8 +808,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, Store.NO));
-      doc.add(new LongField("my_field", value2, Store.NO));
+      doc.add(new LongField("my_field", value, Field.Store.NO));
+      doc.add(new LongField("my_field", value2, Field.Store.NO));
       writer.addDocument(doc);
     }
     final IndexReader reader = DirectoryReader.open(writer);
@@ -891,8 +890,7 @@ public class TestSortOptimization extends LuceneTestCase {
     for (int i = 0; i < numDocs; ++i) {
       final Document doc = new Document();
       final BytesRef value = new BytesRef(Integer.toString(random().nextInt(1000)));
-      doc.add(new StringField("my_field", value, Store.NO));
-      doc.add(new SortedDocValuesField("my_field", value));
+      doc.add(new KeywordField("my_field", value, Field.Store.NO));
       writer.addDocument(doc);
       if (i % 2000 == 0) writer.flush(); // multiple segments
     }
@@ -916,8 +914,7 @@ public class TestSortOptimization extends LuceneTestCase {
       final Document doc = new Document();
       if (random().nextInt(2) == 0) {
         final BytesRef value = new BytesRef(Integer.toString(random().nextInt(1000)));
-        doc.add(new StringField("my_field", value, Store.NO));
-        doc.add(new SortedDocValuesField("my_field", value));
+        doc.add(new KeywordField("my_field", value, Field.Store.NO));
       }
       writer.addDocument(doc);
     }
@@ -936,7 +933,8 @@ public class TestSortOptimization extends LuceneTestCase {
     final int numHits = 5;
 
     { // simple ascending sort
-      SortField sortField = new SortField("my_field", SortField.Type.STRING);
+      SortField sortField =
+          KeywordField.newSortField("my_field", false, SortedSetSelector.Type.MIN);
       sortField.setMissingValue(SortField.STRING_LAST);
       Sort sort = new Sort(sortField);
       TopDocs topDocs = assertSort(reader, sort, numHits, null);
@@ -944,7 +942,7 @@ public class TestSortOptimization extends LuceneTestCase {
     }
 
     { // simple descending sort
-      SortField sortField = new SortField("my_field", SortField.Type.STRING, true);
+      SortField sortField = KeywordField.newSortField("my_field", true, SortedSetSelector.Type.MIN);
       sortField.setMissingValue(SortField.STRING_FIRST);
       Sort sort = new Sort(sortField);
       TopDocs topDocs = assertSort(reader, sort, numHits, null);
@@ -952,21 +950,23 @@ public class TestSortOptimization extends LuceneTestCase {
     }
 
     { // ascending sort that returns missing values first
-      SortField sortField = new SortField("my_field", SortField.Type.STRING);
+      SortField sortField =
+          KeywordField.newSortField("my_field", false, SortedSetSelector.Type.MIN);
       sortField.setMissingValue(SortField.STRING_FIRST);
       Sort sort = new Sort(sortField);
       assertSort(reader, sort, numHits, null);
     }
 
     { // descending sort that returns missing values last
-      SortField sortField = new SortField("my_field", SortField.Type.STRING, true);
+      SortField sortField = KeywordField.newSortField("my_field", true, SortedSetSelector.Type.MIN);
       sortField.setMissingValue(SortField.STRING_LAST);
       Sort sort = new Sort(sortField);
       assertSort(reader, sort, numHits, null);
     }
 
     { // paging ascending sort with after
-      SortField sortField = new SortField("my_field", SortField.Type.STRING);
+      SortField sortField =
+          KeywordField.newSortField("my_field", false, SortedSetSelector.Type.MIN);
       sortField.setMissingValue(SortField.STRING_LAST);
       Sort sort = new Sort(sortField);
       BytesRef afterValue = new BytesRef(random().nextBoolean() ? "23" : "230000000");
@@ -976,7 +976,7 @@ public class TestSortOptimization extends LuceneTestCase {
     }
 
     { // paging descending sort with after
-      SortField sortField = new SortField("my_field", SortField.Type.STRING, true);
+      SortField sortField = KeywordField.newSortField("my_field", true, SortedSetSelector.Type.MIN);
       sortField.setMissingValue(SortField.STRING_FIRST);
       Sort sort = new Sort(sortField);
       BytesRef afterValue = new BytesRef(random().nextBoolean() ? "17" : "170000000");
@@ -986,7 +986,8 @@ public class TestSortOptimization extends LuceneTestCase {
     }
 
     { // paging ascending sort with after that returns missing values first
-      SortField sortField = new SortField("my_field", SortField.Type.STRING);
+      SortField sortField =
+          KeywordField.newSortField("my_field", false, SortedSetSelector.Type.MIN);
       sortField.setMissingValue(SortField.STRING_FIRST);
       Sort sort = new Sort(sortField);
       BytesRef afterValue = new BytesRef(random().nextBoolean() ? "23" : "230000000");
@@ -996,7 +997,7 @@ public class TestSortOptimization extends LuceneTestCase {
     }
 
     { // paging descending sort with after that returns missing values first
-      SortField sortField = new SortField("my_field", SortField.Type.STRING, true);
+      SortField sortField = KeywordField.newSortField("my_field", true, SortedSetSelector.Type.MIN);
       sortField.setMissingValue(SortField.STRING_LAST);
       Sort sort = new Sort(sortField);
       BytesRef afterValue = new BytesRef(random().nextBoolean() ? "17" : "170000000");
@@ -1006,7 +1007,8 @@ public class TestSortOptimization extends LuceneTestCase {
     }
 
     { // test that if there is the secondary sort on _score, hits are still skipped
-      SortField sortField = new SortField("my_field", SortField.Type.STRING);
+      SortField sortField =
+          KeywordField.newSortField("my_field", false, SortedSetSelector.Type.MIN);
       sortField.setMissingValue(SortField.STRING_LAST);
       Sort sort = new Sort(sortField, FIELD_SCORE);
       TopDocs topDocs = assertSort(reader, sort, numHits, null);
@@ -1014,7 +1016,8 @@ public class TestSortOptimization extends LuceneTestCase {
     }
 
     { // test that if string field is a secondary sort, no optimization is run
-      SortField sortField = new SortField("my_field", SortField.Type.STRING);
+      SortField sortField =
+          KeywordField.newSortField("my_field", false, SortedSetSelector.Type.MIN);
       sortField.setMissingValue(SortField.STRING_LAST);
       Sort sort = new Sort(FIELD_SCORE, sortField);
       TopDocs topDocs = assertSort(reader, sort, numHits, null);
@@ -1025,10 +1028,7 @@ public class TestSortOptimization extends LuceneTestCase {
   }
 
   public void doTestStringSortOptimizationDisabled(DirectoryReader reader) throws IOException {
-    SortField sortField =
-        random().nextBoolean()
-            ? new SortedSetSortField("my_field", false)
-            : new SortField("my_field", SortField.Type.STRING);
+    SortField sortField = KeywordField.newSortField("my_field", false, SortedSetSelector.Type.MIN);
     sortField.setMissingValue(SortField.STRING_LAST);
     sortField.setOptimizeSortWithIndexedData(false);
 
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestSortedSetSortField.java b/lucene/core/src/test/org/apache/lucene/search/TestSortedSetSortField.java
index aad6552ccf8..873d948373f 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestSortedSetSortField.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestSortedSetSortField.java
@@ -18,7 +18,7 @@ package org.apache.lucene.search;
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
-import org.apache.lucene.document.SortedSetDocValuesField;
+import org.apache.lucene.document.KeywordField;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.MultiReader;
 import org.apache.lucene.index.Term;
@@ -64,12 +64,12 @@ public class TestSortedSetSortField extends LuceneTestCase {
     Directory dir = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
     Document doc = new Document();
-    doc.add(new SortedSetDocValuesField("value", newBytesRef("baz")));
+    doc.add(new KeywordField("value", newBytesRef("baz"), Field.Store.NO));
     doc.add(newStringField("id", "2", Field.Store.YES));
     writer.addDocument(doc);
     doc = new Document();
-    doc.add(new SortedSetDocValuesField("value", newBytesRef("foo")));
-    doc.add(new SortedSetDocValuesField("value", newBytesRef("bar")));
+    doc.add(new KeywordField("value", newBytesRef("foo"), Field.Store.NO));
+    doc.add(new KeywordField("value", newBytesRef("bar"), Field.Store.NO));
     doc.add(newStringField("id", "1", Field.Store.YES));
     writer.addDocument(doc);
     IndexReader ir = writer.getReader();
@@ -92,12 +92,12 @@ public class TestSortedSetSortField extends LuceneTestCase {
     Directory dir = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
     Document doc = new Document();
-    doc.add(new SortedSetDocValuesField("value", newBytesRef("foo")));
-    doc.add(new SortedSetDocValuesField("value", newBytesRef("bar")));
+    doc.add(new KeywordField("value", newBytesRef("foo"), Field.Store.NO));
+    doc.add(new KeywordField("value", newBytesRef("bar"), Field.Store.NO));
     doc.add(newStringField("id", "1", Field.Store.YES));
     writer.addDocument(doc);
     doc = new Document();
-    doc.add(new SortedSetDocValuesField("value", newBytesRef("baz")));
+    doc.add(new KeywordField("value", newBytesRef("baz"), Field.Store.NO));
     doc.add(newStringField("id", "2", Field.Store.YES));
     writer.addDocument(doc);
 
@@ -121,12 +121,12 @@ public class TestSortedSetSortField extends LuceneTestCase {
     Directory dir = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
     Document doc = new Document();
-    doc.add(new SortedSetDocValuesField("value", newBytesRef("baz")));
+    doc.add(new KeywordField("value", newBytesRef("baz"), Field.Store.NO));
     doc.add(newStringField("id", "2", Field.Store.YES));
     writer.addDocument(doc);
     doc = new Document();
-    doc.add(new SortedSetDocValuesField("value", newBytesRef("foo")));
-    doc.add(new SortedSetDocValuesField("value", newBytesRef("bar")));
+    doc.add(new KeywordField("value", newBytesRef("foo"), Field.Store.NO));
+    doc.add(new KeywordField("value", newBytesRef("bar"), Field.Store.NO));
     doc.add(newStringField("id", "1", Field.Store.YES));
     writer.addDocument(doc);
     doc = new Document();
@@ -156,12 +156,12 @@ public class TestSortedSetSortField extends LuceneTestCase {
     Directory dir = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
     Document doc = new Document();
-    doc.add(new SortedSetDocValuesField("value", newBytesRef("baz")));
+    doc.add(new KeywordField("value", newBytesRef("baz"), Field.Store.NO));
     doc.add(newStringField("id", "2", Field.Store.YES));
     writer.addDocument(doc);
     doc = new Document();
-    doc.add(new SortedSetDocValuesField("value", newBytesRef("foo")));
-    doc.add(new SortedSetDocValuesField("value", newBytesRef("bar")));
+    doc.add(new KeywordField("value", newBytesRef("foo"), Field.Store.NO));
+    doc.add(new KeywordField("value", newBytesRef("bar"), Field.Store.NO));
     doc.add(newStringField("id", "1", Field.Store.YES));
     writer.addDocument(doc);
     doc = new Document();
@@ -191,11 +191,11 @@ public class TestSortedSetSortField extends LuceneTestCase {
     Directory dir = newDirectory();
     RandomIndexWriter writer = new RandomIndexWriter(random(), dir);
     Document doc = new Document();
-    doc.add(new SortedSetDocValuesField("value", newBytesRef("baz")));
+    doc.add(new KeywordField("value", newBytesRef("baz"), Field.Store.NO));
     doc.add(newStringField("id", "2", Field.Store.YES));
     writer.addDocument(doc);
     doc = new Document();
-    doc.add(new SortedSetDocValuesField("value", newBytesRef("bar")));
+    doc.add(new KeywordField("value", newBytesRef("bar"), Field.Store.NO));
     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 7b172d65d41..9c683d3937c 100644
--- a/lucene/demo/src/java/org/apache/lucene/demo/IndexFiles.java
+++ b/lucene/demo/src/java/org/apache/lucene/demo/IndexFiles.java
@@ -34,9 +34,9 @@ import org.apache.lucene.demo.knn.DemoEmbeddings;
 import org.apache.lucene.demo.knn.KnnVectorDict;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
+import org.apache.lucene.document.KeywordField;
 import org.apache.lucene.document.KnnFloatVectorField;
 import org.apache.lucene.document.LongField;
-import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
@@ -234,8 +234,7 @@ public class IndexFiles implements AutoCloseable {
       // field that is indexed (i.e. searchable), but don't tokenize
       // the field into separate words and don't index term frequency
       // or positional information:
-      Field pathField = new StringField("path", file.toString(), Field.Store.YES);
-      doc.add(pathField);
+      doc.add(new KeywordField("path", file.toString(), Field.Store.YES));
 
       // Add the last modified date of the file a field named "modified".
       // Use a LongField that is indexed with points and doc values, and is efficient