You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2022/04/04 16:21:10 UTC

[GitHub] [lucene] jpountz commented on a diff in pull request #767: LUCENE-10436: Deprecate DocValuesFieldExistsQuery, NormsFieldExistsQuery and KnnVectorFieldExistsQuery with FieldExistsQuery

jpountz commented on code in PR #767:
URL: https://github.com/apache/lucene/pull/767#discussion_r841919931


##########
lucene/core/src/test/org/apache/lucene/search/TestFieldExistsQuery.java:
##########
@@ -0,0 +1,717 @@
+/*
+ * 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.search;
+
+import java.io.IOException;
+import org.apache.lucene.document.BinaryPoint;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoubleDocValuesField;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.KnnVectorField;
+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.StringField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.NoMergePolicy;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.tests.index.RandomIndexWriter;
+import org.apache.lucene.tests.util.LuceneTestCase;
+import org.apache.lucene.tests.util.TestUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.VectorUtil;
+
+public class TestFieldExistsQuery extends LuceneTestCase {
+
+  public void testDocValuesRewriteWithTermsPresent() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    final int numDocs = atLeast(100);
+    for (int i = 0; i < numDocs; ++i) {
+      Document doc = new Document();
+      doc.add(new DoubleDocValuesField("f", 2.0));
+      doc.add(new StringField("f", random().nextBoolean() ? "yes" : "no", Store.NO));
+      iw.addDocument(doc);
+    }
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    iw.close();
+
+    assertTrue((new FieldExistsQuery("f")).rewrite(reader) instanceof MatchAllDocsQuery);
+    reader.close();
+    dir.close();
+  }
+
+  public void testDocValuesRewriteWithPointValuesPresent() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    final int numDocs = atLeast(100);
+    for (int i = 0; i < numDocs; ++i) {
+      Document doc = new Document();
+      doc.add(new BinaryPoint("dim", new byte[4], new byte[4]));
+      iw.addDocument(doc);
+    }
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    iw.close();
+
+    expectThrows(IllegalStateException.class, () -> new FieldExistsQuery("dim").rewrite(reader));

Review Comment:
   can you fix the indexing logic to index a field with doc values instead, and keep checking that the query rewrites to a MatchAllDocsQuery?



##########
lucene/core/src/test/org/apache/lucene/search/TestFieldExistsQuery.java:
##########
@@ -0,0 +1,717 @@
+/*
+ * 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.search;
+
+import java.io.IOException;
+import org.apache.lucene.document.BinaryPoint;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoubleDocValuesField;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.KnnVectorField;
+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.StringField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.NoMergePolicy;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.tests.index.RandomIndexWriter;
+import org.apache.lucene.tests.util.LuceneTestCase;
+import org.apache.lucene.tests.util.TestUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.VectorUtil;
+
+public class TestFieldExistsQuery extends LuceneTestCase {
+
+  public void testDocValuesRewriteWithTermsPresent() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    final int numDocs = atLeast(100);
+    for (int i = 0; i < numDocs; ++i) {
+      Document doc = new Document();
+      doc.add(new DoubleDocValuesField("f", 2.0));
+      doc.add(new StringField("f", random().nextBoolean() ? "yes" : "no", Store.NO));
+      iw.addDocument(doc);
+    }
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    iw.close();
+
+    assertTrue((new FieldExistsQuery("f")).rewrite(reader) instanceof MatchAllDocsQuery);
+    reader.close();
+    dir.close();
+  }
+
+  public void testDocValuesRewriteWithPointValuesPresent() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    final int numDocs = atLeast(100);
+    for (int i = 0; i < numDocs; ++i) {
+      Document doc = new Document();
+      doc.add(new BinaryPoint("dim", new byte[4], new byte[4]));
+      iw.addDocument(doc);
+    }
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    iw.close();
+
+    expectThrows(IllegalStateException.class, () -> new FieldExistsQuery("dim").rewrite(reader));
+    reader.close();
+    dir.close();
+  }
+
+  public void testDocValuesNoRewrite() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    final int numDocs = atLeast(100);
+    for (int i = 0; i < numDocs; ++i) {
+      Document doc = new Document();
+      doc.add(new DoubleDocValuesField("dim", 2.0));
+      iw.addDocument(doc);
+    }
+    for (int i = 0; i < numDocs; ++i) {
+      Document doc = new Document();
+      doc.add(new StringField("f", random().nextBoolean() ? "yes" : "no", Store.NO));
+      iw.addDocument(doc);
+    }
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    iw.close();
+
+    assertFalse((new FieldExistsQuery("dim")).rewrite(reader) instanceof MatchAllDocsQuery);
+    expectThrows(IllegalStateException.class, () -> new FieldExistsQuery("f").rewrite(reader));
+    reader.close();
+    dir.close();
+  }
+
+  public void testDocValuesNoRewriteWithDocValues() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    final int numDocs = atLeast(100);
+    for (int i = 0; i < numDocs; ++i) {
+      Document doc = new Document();
+      doc.add(new NumericDocValuesField("dv1", 1));
+      doc.add(new SortedNumericDocValuesField("dv2", 1));
+      doc.add(new SortedNumericDocValuesField("dv2", 2));
+      iw.addDocument(doc);
+    }
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    iw.close();
+
+    assertFalse((new FieldExistsQuery("dv1")).rewrite(reader) instanceof MatchAllDocsQuery);
+    assertFalse((new FieldExistsQuery("dv2")).rewrite(reader) instanceof MatchAllDocsQuery);
+    assertFalse((new FieldExistsQuery("dv3")).rewrite(reader) instanceof MatchAllDocsQuery);
+    reader.close();
+    dir.close();
+  }
+
+  public void testDocValuesRandom() throws IOException {
+    final int iters = atLeast(10);
+    for (int iter = 0; iter < iters; ++iter) {
+      Directory dir = newDirectory();
+      RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+      final int numDocs = atLeast(100);
+      for (int i = 0; i < numDocs; ++i) {
+        Document doc = new Document();
+        final boolean hasValue = random().nextBoolean();
+        if (hasValue) {
+          doc.add(new NumericDocValuesField("dv1", 1));
+          doc.add(new SortedNumericDocValuesField("dv2", 1));
+          doc.add(new SortedNumericDocValuesField("dv2", 2));
+          doc.add(new StringField("has_value", "yes", Store.NO));
+        }
+        doc.add(new StringField("f", random().nextBoolean() ? "yes" : "no", Store.NO));
+        iw.addDocument(doc);
+      }
+      if (random().nextBoolean()) {
+        iw.deleteDocuments(new TermQuery(new Term("f", "no")));
+      }
+      iw.commit();
+      final IndexReader reader = iw.getReader();
+      final IndexSearcher searcher = newSearcher(reader);
+      iw.close();
+
+      assertSameMatches(
+          searcher,
+          new TermQuery(new Term("has_value", "yes")),
+          new FieldExistsQuery("dv1"),
+          false);
+      assertSameMatches(
+          searcher,
+          new TermQuery(new Term("has_value", "yes")),
+          new FieldExistsQuery("dv2"),
+          false);
+
+      reader.close();
+      dir.close();
+    }
+  }
+
+  public void testDocValuesApproximation() throws IOException {
+    final int iters = atLeast(10);
+    for (int iter = 0; iter < iters; ++iter) {
+      Directory dir = newDirectory();
+      RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+      final int numDocs = atLeast(100);
+      for (int i = 0; i < numDocs; ++i) {
+        Document doc = new Document();
+        final boolean hasValue = random().nextBoolean();
+        if (hasValue) {
+          doc.add(new NumericDocValuesField("dv1", 1));
+          doc.add(new SortedNumericDocValuesField("dv2", 1));
+          doc.add(new SortedNumericDocValuesField("dv2", 2));
+          doc.add(new StringField("has_value", "yes", Store.NO));
+        }
+        doc.add(new StringField("f", random().nextBoolean() ? "yes" : "no", Store.NO));
+        iw.addDocument(doc);
+      }
+      if (random().nextBoolean()) {
+        iw.deleteDocuments(new TermQuery(new Term("f", "no")));
+      }
+      iw.commit();
+      final IndexReader reader = iw.getReader();
+      final IndexSearcher searcher = newSearcher(reader);
+      iw.close();
+
+      BooleanQuery.Builder ref = new BooleanQuery.Builder();
+      ref.add(new TermQuery(new Term("f", "yes")), Occur.MUST);
+      ref.add(new TermQuery(new Term("has_value", "yes")), Occur.FILTER);
+
+      BooleanQuery.Builder bq1 = new BooleanQuery.Builder();
+      bq1.add(new TermQuery(new Term("f", "yes")), Occur.MUST);
+      bq1.add(new FieldExistsQuery("dv1"), Occur.FILTER);
+      assertSameMatches(searcher, ref.build(), bq1.build(), true);
+
+      BooleanQuery.Builder bq2 = new BooleanQuery.Builder();
+      bq2.add(new TermQuery(new Term("f", "yes")), Occur.MUST);
+      bq2.add(new FieldExistsQuery("dv2"), Occur.FILTER);
+      assertSameMatches(searcher, ref.build(), bq2.build(), true);
+
+      reader.close();
+      dir.close();
+    }
+  }
+
+  public void testDocValuesScore() throws IOException {
+    final int iters = atLeast(10);
+    for (int iter = 0; iter < iters; ++iter) {
+      Directory dir = newDirectory();
+      RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+      final int numDocs = atLeast(100);
+      for (int i = 0; i < numDocs; ++i) {
+        Document doc = new Document();
+        final boolean hasValue = random().nextBoolean();
+        if (hasValue) {
+          doc.add(new NumericDocValuesField("dv1", 1));
+          doc.add(new SortedNumericDocValuesField("dv2", 1));
+          doc.add(new SortedNumericDocValuesField("dv2", 2));
+          doc.add(new StringField("has_value", "yes", Store.NO));
+        }
+        doc.add(new StringField("f", random().nextBoolean() ? "yes" : "no", Store.NO));
+        iw.addDocument(doc);
+      }
+      if (random().nextBoolean()) {
+        iw.deleteDocuments(new TermQuery(new Term("f", "no")));
+      }
+      iw.commit();
+      final IndexReader reader = iw.getReader();
+      final IndexSearcher searcher = newSearcher(reader);
+      iw.close();
+
+      final float boost = random().nextFloat() * 10;
+      final Query ref =
+          new BoostQuery(
+              new ConstantScoreQuery(new TermQuery(new Term("has_value", "yes"))), boost);
+
+      final Query q1 = new BoostQuery(new FieldExistsQuery("dv1"), boost);
+      assertSameMatches(searcher, ref, q1, true);
+
+      final Query q2 = new BoostQuery(new FieldExistsQuery("dv2"), boost);
+      assertSameMatches(searcher, ref, q2, true);
+
+      reader.close();
+      dir.close();
+    }
+  }
+
+  public void testDocValuesMissingField() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    iw.addDocument(new Document());
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    final IndexSearcher searcher = newSearcher(reader);
+    iw.close();
+    assertEquals(0, searcher.count(new FieldExistsQuery("f")));
+    reader.close();
+    dir.close();
+  }
+
+  public void testDocValuesAllDocsHaveField() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    doc.add(new NumericDocValuesField("f", 1));
+    iw.addDocument(doc);
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    final IndexSearcher searcher = newSearcher(reader);
+    iw.close();
+    assertEquals(1, searcher.count(new FieldExistsQuery("f")));
+    reader.close();
+    dir.close();
+  }
+
+  public void testDocValuesFieldExistsButNoDocsHaveField() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    // 1st segment has the field, but 2nd one does not
+    Document doc = new Document();
+    doc.add(new NumericDocValuesField("f", 1));
+    iw.addDocument(doc);
+    iw.commit();
+    iw.addDocument(new Document());
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    final IndexSearcher searcher = newSearcher(reader);
+    iw.close();
+    assertEquals(1, searcher.count(new FieldExistsQuery("f")));
+    reader.close();
+    dir.close();
+  }
+
+  public void testDocValuesQueryMatchesCount() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+    int randomNumDocs = TestUtil.nextInt(random(), 10, 100);
+    int numMatchingDocs = 0;
+
+    for (int i = 0; i < randomNumDocs; i++) {
+      Document doc = new Document();
+      // ensure we index at least a document with long between 0 and 10
+      if (i == 0 || random().nextBoolean()) {
+        doc.add(new LongPoint("long", i));
+        doc.add(new NumericDocValuesField("long", i));
+        doc.add(new StringField("string", "value", Store.NO));
+        doc.add(new SortedDocValuesField("string", new BytesRef("value")));
+        numMatchingDocs++;
+      }
+      w.addDocument(doc);
+    }
+    w.forceMerge(1);
+
+    DirectoryReader reader = w.getReader();
+    final IndexSearcher searcher = new IndexSearcher(reader);
+
+    assertSameCount(reader, searcher, "long", numMatchingDocs);
+    assertSameCount(reader, searcher, "string", numMatchingDocs);
+    assertSameCount(reader, searcher, "doesNotExist", 0);
+
+    // Test that we can't count in O(1) when there are deleted documents
+    w.w.getConfig().setMergePolicy(NoMergePolicy.INSTANCE);
+    w.deleteDocuments(LongPoint.newRangeQuery("long", 0L, 10L));
+    DirectoryReader reader2 = w.getReader();
+    final IndexSearcher searcher2 = new IndexSearcher(reader2);
+    final Query testQuery = new FieldExistsQuery("long");
+    final Weight weight2 = searcher2.createWeight(testQuery, ScoreMode.COMPLETE, 1);
+    assertEquals(weight2.count(reader2.leaves().get(0)), -1);
+
+    IOUtils.close(reader, reader2, w, dir);
+  }
+
+  public void testNormsRandom() throws IOException {
+    final int iters = atLeast(10);
+    for (int iter = 0; iter < iters; ++iter) {
+      Directory dir = newDirectory();
+      RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+      final int numDocs = atLeast(100);
+      for (int i = 0; i < numDocs; ++i) {
+        Document doc = new Document();
+        final boolean hasValue = random().nextBoolean();
+        if (hasValue) {
+          doc.add(new TextField("text1", "value", Store.NO));
+          doc.add(new StringField("has_value", "yes", Store.NO));
+        }
+        doc.add(new StringField("f", random().nextBoolean() ? "yes" : "no", Store.NO));
+        iw.addDocument(doc);
+      }
+      if (random().nextBoolean()) {
+        iw.deleteDocuments(new TermQuery(new Term("f", "no")));
+      }
+      iw.commit();
+      final IndexReader reader = iw.getReader();
+      final IndexSearcher searcher = newSearcher(reader);
+      iw.close();
+
+      assertSameMatches(
+          searcher,
+          new TermQuery(new Term("has_value", "yes")),
+          new FieldExistsQuery("text1"),
+          false);
+
+      reader.close();
+      dir.close();
+    }
+  }
+
+  public void testNormsApproximation() throws IOException {
+    final int iters = atLeast(10);
+    for (int iter = 0; iter < iters; ++iter) {
+      Directory dir = newDirectory();
+      RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+      final int numDocs = atLeast(100);
+      for (int i = 0; i < numDocs; ++i) {
+        Document doc = new Document();
+        final boolean hasValue = random().nextBoolean();
+        if (hasValue) {
+          doc.add(new TextField("text1", "value", Store.NO));
+          doc.add(new StringField("has_value", "yes", Store.NO));
+        }
+        doc.add(new StringField("f", random().nextBoolean() ? "yes" : "no", Store.NO));
+        iw.addDocument(doc);
+      }
+      if (random().nextBoolean()) {
+        iw.deleteDocuments(new TermQuery(new Term("f", "no")));
+      }
+      iw.commit();
+      final IndexReader reader = iw.getReader();
+      final IndexSearcher searcher = newSearcher(reader);
+      iw.close();
+
+      BooleanQuery.Builder ref = new BooleanQuery.Builder();
+      ref.add(new TermQuery(new Term("f", "yes")), Occur.MUST);
+      ref.add(new TermQuery(new Term("has_value", "yes")), Occur.FILTER);
+
+      BooleanQuery.Builder bq1 = new BooleanQuery.Builder();
+      bq1.add(new TermQuery(new Term("f", "yes")), Occur.MUST);
+      bq1.add(new FieldExistsQuery("text1"), Occur.FILTER);
+      assertSameMatches(searcher, ref.build(), bq1.build(), true);
+
+      reader.close();
+      dir.close();
+    }
+  }
+
+  public void testNormsScore() throws IOException {
+    final int iters = atLeast(10);
+    for (int iter = 0; iter < iters; ++iter) {
+      Directory dir = newDirectory();
+      RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+      final int numDocs = atLeast(100);
+      for (int i = 0; i < numDocs; ++i) {
+        Document doc = new Document();
+        final boolean hasValue = random().nextBoolean();
+        if (hasValue) {
+          doc.add(new TextField("text1", "value", Store.NO));
+          doc.add(new StringField("has_value", "yes", Store.NO));
+        }
+        doc.add(new StringField("f", random().nextBoolean() ? "yes" : "no", Store.NO));
+        iw.addDocument(doc);
+      }
+      if (random().nextBoolean()) {
+        iw.deleteDocuments(new TermQuery(new Term("f", "no")));
+      }
+      iw.commit();
+      final IndexReader reader = iw.getReader();
+      final IndexSearcher searcher = newSearcher(reader);
+      iw.close();
+
+      final float boost = random().nextFloat() * 10;
+      final Query ref =
+          new BoostQuery(
+              new ConstantScoreQuery(new TermQuery(new Term("has_value", "yes"))), boost);
+
+      final Query q1 = new BoostQuery(new FieldExistsQuery("text1"), boost);
+      assertSameMatches(searcher, ref, q1, true);
+
+      reader.close();
+      dir.close();
+    }
+  }
+
+  public void testNormsMissingField() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    iw.addDocument(new Document());
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    final IndexSearcher searcher = newSearcher(reader);
+    iw.close();
+    assertEquals(0, searcher.count(new FieldExistsQuery("f")));
+    reader.close();
+    dir.close();
+  }
+
+  public void testNormsAllDocsHaveField() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    Document doc = new Document();
+    doc.add(new TextField("f", "value", Store.NO));
+    iw.addDocument(doc);
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    final IndexSearcher searcher = newSearcher(reader);
+    iw.close();
+    assertEquals(1, searcher.count(new FieldExistsQuery("f")));
+    reader.close();
+    dir.close();
+  }
+
+  public void testNormsFieldExistsButNoDocsHaveField() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    // 1st segment has the field, but 2nd one does not
+    Document doc = new Document();
+    doc.add(new TextField("f", "value", Store.NO));
+    iw.addDocument(doc);
+    iw.commit();
+    iw.addDocument(new Document());
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    final IndexSearcher searcher = newSearcher(reader);
+    iw.close();
+    assertEquals(1, searcher.count(new FieldExistsQuery("f")));
+    reader.close();
+    dir.close();
+  }
+
+  public void testNormsQueryMatchesCount() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+    int randomNumDocs = TestUtil.nextInt(random(), 10, 100);
+
+    FieldType noNormsFieldType = new FieldType();
+    noNormsFieldType.setOmitNorms(true);
+    noNormsFieldType.setIndexOptions(IndexOptions.DOCS);
+
+    Document doc = new Document();
+    doc.add(new TextField("text", "always here", Store.NO));
+    doc.add(new TextField("text_s", "", Store.NO));
+    doc.add(new Field("text_n", "always here", noNormsFieldType));
+    w.addDocument(doc);
+
+    for (int i = 1; i < randomNumDocs; i++) {
+      doc.clear();
+      doc.add(new TextField("text", "some text", Store.NO));
+      doc.add(new TextField("text_s", "some text", Store.NO));
+      doc.add(new Field("text_n", "some here", noNormsFieldType));
+      w.addDocument(doc);
+    }
+    w.forceMerge(1);
+
+    DirectoryReader reader = w.getReader();
+    final IndexSearcher searcher = new IndexSearcher(reader);
+
+    assertNormsCountWithShortcut(searcher, "text", randomNumDocs);
+    assertNormsCountWithShortcut(searcher, "doesNotExist", 0);
+    expectThrows(IllegalStateException.class, () -> searcher.count(new FieldExistsQuery("text_n")));

Review Comment:
   checking for an exception here makes sense to me however :+1:



##########
lucene/core/src/test/org/apache/lucene/search/TestFieldExistsQuery.java:
##########
@@ -0,0 +1,717 @@
+/*
+ * 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.search;
+
+import java.io.IOException;
+import org.apache.lucene.document.BinaryPoint;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoubleDocValuesField;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.KnnVectorField;
+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.StringField;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.NoMergePolicy;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.tests.index.RandomIndexWriter;
+import org.apache.lucene.tests.util.LuceneTestCase;
+import org.apache.lucene.tests.util.TestUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.VectorUtil;
+
+public class TestFieldExistsQuery extends LuceneTestCase {
+
+  public void testDocValuesRewriteWithTermsPresent() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    final int numDocs = atLeast(100);
+    for (int i = 0; i < numDocs; ++i) {
+      Document doc = new Document();
+      doc.add(new DoubleDocValuesField("f", 2.0));
+      doc.add(new StringField("f", random().nextBoolean() ? "yes" : "no", Store.NO));
+      iw.addDocument(doc);
+    }
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    iw.close();
+
+    assertTrue((new FieldExistsQuery("f")).rewrite(reader) instanceof MatchAllDocsQuery);
+    reader.close();
+    dir.close();
+  }
+
+  public void testDocValuesRewriteWithPointValuesPresent() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    final int numDocs = atLeast(100);
+    for (int i = 0; i < numDocs; ++i) {
+      Document doc = new Document();
+      doc.add(new BinaryPoint("dim", new byte[4], new byte[4]));
+      iw.addDocument(doc);
+    }
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    iw.close();
+
+    expectThrows(IllegalStateException.class, () -> new FieldExistsQuery("dim").rewrite(reader));
+    reader.close();
+    dir.close();
+  }
+
+  public void testDocValuesNoRewrite() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
+    final int numDocs = atLeast(100);
+    for (int i = 0; i < numDocs; ++i) {
+      Document doc = new Document();
+      doc.add(new DoubleDocValuesField("dim", 2.0));
+      iw.addDocument(doc);
+    }
+    for (int i = 0; i < numDocs; ++i) {
+      Document doc = new Document();
+      doc.add(new StringField("f", random().nextBoolean() ? "yes" : "no", Store.NO));
+      iw.addDocument(doc);
+    }
+    iw.commit();
+    final IndexReader reader = iw.getReader();
+    iw.close();
+
+    assertFalse((new FieldExistsQuery("dim")).rewrite(reader) instanceof MatchAllDocsQuery);
+    expectThrows(IllegalStateException.class, () -> new FieldExistsQuery("f").rewrite(reader));

Review Comment:
   And likewise here, can you index points in addition to doc values for `dim` and doc values in addition to terms for `f`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org