You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2021/05/03 15:18:48 UTC

[lucene] branch main updated: LUCENE-9948: Automatically detect multi- vs. single-valued cases in LongValueFacetCounts (#122)

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

rmuir 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 650cad1  LUCENE-9948: Automatically detect multi- vs. single-valued cases in LongValueFacetCounts (#122)
650cad1 is described below

commit 650cad19a2ccf780a8ce30a6a8dfcf0d9d85022e
Author: Greg Miller <gs...@gmail.com>
AuthorDate: Mon May 3 08:18:38 2021 -0700

    LUCENE-9948: Automatically detect multi- vs. single-valued cases in LongValueFacetCounts (#122)
    
    The public API in LongValueFacetCounts previously required the user to specify whether-or-not a field being counted should be single- or multi-valued (i.e., is it NumericDocValues or SortedNumericDocValues). Since we can detect this automatically, it seems unnecessary to ask users to specify.
    
    Co-authored-by: Greg Miller <gm...@amazon.com>
---
 lucene/CHANGES.txt                                 |   5 +-
 lucene/MIGRATE.md                                  |   6 +
 .../apache/lucene/facet/LongValueFacetCounts.java  | 168 +++++++--------------
 .../lucene/facet/TestLongValueFacetCounts.java     |  40 +++--
 4 files changed, 86 insertions(+), 133 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 61fbee6..45dde72 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -110,7 +110,10 @@ API Changes
   Adrien Grand, Simon Willnauer)
 
 * LUCENE-9047: Directory API is now little endian. (Ignacio Vera, Adrien Grand)  
-  
+
+* LUCENE-9948: No longer require the user to specify whether-or-not a field is multi-valued in
+  LongValueFacetCounts (detect automatically based on what is indexed). (Greg Miller)
+
 Improvements
 
 * LUCENE-9687: Hunspell support improvements: add API for spell-checking and suggestions, support compound words,
diff --git a/lucene/MIGRATE.md b/lucene/MIGRATE.md
index 783efff..29f5f7b 100644
--- a/lucene/MIGRATE.md
+++ b/lucene/MIGRATE.md
@@ -400,3 +400,9 @@ field.
 
 Lucene index readers are now using so little memory with the default codec that
 it was decided to remove the ability to estimate their RAM usage.
+
+## LongValueFacetCounts no longer accepts multiValued param in constructors (LUCENE-9948)
+
+LongValueFacetCounts will now automatically detect whether-or-not an indexed field is single- or
+multi-valued. The user no longer needs to provide this information to the ctors. Migrating should
+be as simple as no longer providing this boolean.
\ No newline at end of file
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/LongValueFacetCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/LongValueFacetCounts.java
index 4379ab7..520d931 100644
--- a/lucene/facet/src/java/org/apache/lucene/facet/LongValueFacetCounts.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/LongValueFacetCounts.java
@@ -54,52 +54,35 @@ public class LongValueFacetCounts extends Facets {
   /** Used for all values that are >= 1K. */
   private final LongIntScatterMap hashCounts = new LongIntScatterMap();
 
+  /** Field being counted. */
   private final String field;
 
   /**
-   * Total number of values counted, which is the subset of hits that had a value for this field.
+   * Total value count. For single-value cases, this is the subset of hits that had a value for this
+   * field.
    */
   private int totCount;
 
   /**
    * Create {@code LongValueFacetCounts}, using either single-valued {@link NumericDocValues} or
-   * multi-valued {@link SortedNumericDocValues} from the specified field.
+   * multi-valued {@link SortedNumericDocValues} from the specified field (depending on what has
+   * been indexed).
    */
-  public LongValueFacetCounts(String field, FacetsCollector hits, boolean multiValued)
-      throws IOException {
-    this(field, null, hits, multiValued);
+  public LongValueFacetCounts(String field, FacetsCollector hits) throws IOException {
+    this(field, null, hits);
   }
 
   /**
-   * Create {@code LongValueFacetCounts}, using the provided {@link LongValuesSource}. If hits is
-   * null then all facets are counted.
+   * Create {@code LongValueFacetCounts}, using the provided {@link LongValuesSource} if non-null.
+   * If {@code valueSource} is null, doc values from the provided {@code field} will be used.
    */
   public LongValueFacetCounts(String field, LongValuesSource valueSource, FacetsCollector hits)
       throws IOException {
-    this(field, valueSource, hits, false);
-  }
-
-  /**
-   * Create {@code LongValueFacetCounts}, using the provided {@link LongValuesSource}. random access
-   * (implement {@link org.apache.lucene.search.DocIdSet#bits}).
-   */
-  public LongValueFacetCounts(
-      String field, LongValuesSource valueSource, FacetsCollector hits, boolean multiValued)
-      throws IOException {
     this.field = field;
-    if (valueSource == null) {
-      if (multiValued) {
-        countMultiValued(field, hits.getMatchingDocs());
-      } else {
-        count(field, hits.getMatchingDocs());
-      }
-    } else {
-      // value source is always single valued
-      if (multiValued) {
-        throw new IllegalArgumentException(
-            "can only compute multi-valued facets directly from doc values (when valueSource is null)");
-      }
+    if (valueSource != null) {
       count(valueSource, hits.getMatchingDocs());
+    } else {
+      count(field, hits.getMatchingDocs());
     }
   }
 
@@ -107,31 +90,32 @@ public class LongValueFacetCounts extends Facets {
    * Counts all facet values for this reader. This produces the same result as computing facets on a
    * {@link org.apache.lucene.search.MatchAllDocsQuery}, but is more efficient.
    */
-  public LongValueFacetCounts(String field, IndexReader reader, boolean multiValued)
-      throws IOException {
-    this.field = field;
-    if (multiValued) {
-      countAllMultiValued(reader, field);
-    } else {
-      countAll(reader, field);
-    }
+  public LongValueFacetCounts(String field, IndexReader reader) throws IOException {
+    this(field, null, reader);
   }
 
   /**
-   * Counts all facet values for the provided {@link LongValuesSource}. This produces the same
-   * result as computing facets on a {@link org.apache.lucene.search.MatchAllDocsQuery}, but is more
-   * efficient.
+   * Counts all facet values for the provided {@link LongValuesSource} if non-null. If {@code
+   * valueSource} is null, doc values from the provided {@code field} will be used. This produces
+   * the same result as computing facets on a {@link org.apache.lucene.search.MatchAllDocsQuery},
+   * but is more efficient.
    */
   public LongValueFacetCounts(String field, LongValuesSource valueSource, IndexReader reader)
       throws IOException {
     this.field = field;
-    countAll(valueSource, field, reader);
+    if (valueSource != null) {
+      countAll(reader, valueSource);
+    } else {
+      countAll(reader, field);
+    }
   }
 
+  /** Counts from the provided valueSource. */
   private void count(LongValuesSource valueSource, List<MatchingDocs> matchingDocs)
       throws IOException {
 
     for (MatchingDocs hits : matchingDocs) {
+
       LongValues fv = valueSource.getValues(hits.context, null);
 
       // NOTE: this is not as efficient as working directly with the doc values APIs in the sparse
@@ -152,80 +136,41 @@ public class LongValueFacetCounts extends Facets {
     }
   }
 
+  /** Counts from the field's indexed doc values. */
   private void count(String field, List<MatchingDocs> matchingDocs) throws IOException {
-    for (MatchingDocs hits : matchingDocs) {
-      NumericDocValues fv = hits.context.reader().getNumericDocValues(field);
-      if (fv == null) {
-        continue;
-      }
-      countOneSegment(fv, hits);
-    }
-  }
-
-  private void countOneSegment(NumericDocValues values, MatchingDocs hits) throws IOException {
-    DocIdSetIterator it =
-        ConjunctionDISI.intersectIterators(Arrays.asList(hits.bits.iterator(), values));
-
-    for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
-      increment(values.longValue());
-      totCount++;
-    }
-  }
-
-  /** Counts directly from SortedNumericDocValues. */
-  private void countMultiValued(String field, List<MatchingDocs> matchingDocs) throws IOException {
 
     for (MatchingDocs hits : matchingDocs) {
-      SortedNumericDocValues values = hits.context.reader().getSortedNumericDocValues(field);
-      if (values == null) {
-        // this field has no doc values for this segment
-        continue;
-      }
 
-      NumericDocValues singleValues = DocValues.unwrapSingleton(values);
+      SortedNumericDocValues multiValues = DocValues.getSortedNumeric(hits.context.reader(), field);
+      NumericDocValues singleValues = DocValues.unwrapSingleton(multiValues);
 
       if (singleValues != null) {
-        countOneSegment(singleValues, hits);
+
+        DocIdSetIterator it =
+            ConjunctionDISI.intersectIterators(Arrays.asList(hits.bits.iterator(), singleValues));
+
+        for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+          increment(singleValues.longValue());
+          totCount++;
+        }
       } else {
 
         DocIdSetIterator it =
-            ConjunctionDISI.intersectIterators(Arrays.asList(hits.bits.iterator(), values));
+            ConjunctionDISI.intersectIterators(Arrays.asList(hits.bits.iterator(), multiValues));
 
         for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
-          int limit = values.docValueCount();
+          int limit = multiValues.docValueCount();
           totCount += limit;
           for (int i = 0; i < limit; i++) {
-            increment(values.nextValue());
+            increment(multiValues.nextValue());
           }
         }
       }
     }
   }
 
-  /** Optimized version that directly counts all doc values. */
-  private void countAll(IndexReader reader, String field) throws IOException {
-
-    for (LeafReaderContext context : reader.leaves()) {
-
-      NumericDocValues values = context.reader().getNumericDocValues(field);
-      if (values == null) {
-        // this field has no doc values for this segment
-        continue;
-      }
-
-      countAllOneSegment(values);
-    }
-  }
-
-  private void countAllOneSegment(NumericDocValues values) throws IOException {
-    while (values.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
-      totCount++;
-      increment(values.longValue());
-    }
-  }
-
-  private void countAll(LongValuesSource valueSource, String field, IndexReader reader)
-      throws IOException {
+  /** Count everything in the provided valueSource. */
+  private void countAll(IndexReader reader, LongValuesSource valueSource) throws IOException {
 
     for (LeafReaderContext context : reader.leaves()) {
       LongValues fv = valueSource.getValues(context, null);
@@ -241,24 +186,27 @@ public class LongValueFacetCounts extends Facets {
     }
   }
 
-  private void countAllMultiValued(IndexReader reader, String field) throws IOException {
+  /** Count everything in the specified field. */
+  private void countAll(IndexReader reader, String field) throws IOException {
 
     for (LeafReaderContext context : reader.leaves()) {
 
-      SortedNumericDocValues values = context.reader().getSortedNumericDocValues(field);
-      if (values == null) {
-        // this field has no doc values for this segment
-        continue;
-      }
-      NumericDocValues singleValues = DocValues.unwrapSingleton(values);
+      SortedNumericDocValues multiValues = DocValues.getSortedNumeric(context.reader(), field);
+      NumericDocValues singleValues = DocValues.unwrapSingleton(multiValues);
+
       if (singleValues != null) {
-        countAllOneSegment(singleValues);
+
+        while (singleValues.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
+          totCount++;
+          increment(singleValues.longValue());
+        }
       } else {
-        while (values.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
-          int limit = values.docValueCount();
+
+        while (multiValues.nextDoc() != DocIdSetIterator.NO_MORE_DOCS) {
+          int limit = multiValues.docValueCount();
           totCount += limit;
           for (int i = 0; i < limit; i++) {
-            increment(values.nextValue());
+            increment(multiValues.nextValue());
           }
         }
       }
@@ -294,7 +242,7 @@ public class LongValueFacetCounts extends Facets {
   /** Returns the specified top number of facets, sorted by count. */
   public FacetResult getTopChildrenSortByCount(int topN) {
     PriorityQueue<Entry> pq =
-        new PriorityQueue<Entry>(Math.min(topN, counts.length + hashCounts.size())) {
+        new PriorityQueue<>(Math.min(topN, counts.length + hashCounts.size())) {
           @Override
           protected boolean lessThan(Entry a, Entry b) {
             // sort by count descending, breaking ties by value ascending:
@@ -409,13 +357,13 @@ public class LongValueFacetCounts extends Facets {
   }
 
   @Override
-  public Number getSpecificValue(String dim, String... path) throws IOException {
+  public Number getSpecificValue(String dim, String... path) {
     // TODO: should we impl this?
     throw new UnsupportedOperationException();
   }
 
   @Override
-  public List<FacetResult> getAllDims(int topN) throws IOException {
+  public List<FacetResult> getAllDims(int topN) {
     return Collections.singletonList(getTopChildren(topN, field));
   }
 
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/TestLongValueFacetCounts.java b/lucene/facet/src/test/org/apache/lucene/facet/TestLongValueFacetCounts.java
index 1c4b5eb..5821052 100644
--- a/lucene/facet/src/test/org/apache/lucene/facet/TestLongValueFacetCounts.java
+++ b/lucene/facet/src/test/org/apache/lucene/facet/TestLongValueFacetCounts.java
@@ -19,7 +19,7 @@ package org.apache.lucene.facet;
 
 import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -60,7 +60,7 @@ public class TestLongValueFacetCounts extends LuceneTestCase {
     IndexSearcher s = newSearcher(r);
     s.search(new MatchAllDocsQuery(), fc);
 
-    LongValueFacetCounts facets = new LongValueFacetCounts("field", fc, false);
+    LongValueFacetCounts facets = new LongValueFacetCounts("field", fc);
 
     FacetResult result = facets.getAllChildrenSortByValue();
     assertEquals(
@@ -87,7 +87,7 @@ public class TestLongValueFacetCounts extends LuceneTestCase {
     IndexSearcher s = newSearcher(r);
     s.search(new MatchAllDocsQuery(), fc);
 
-    LongValueFacetCounts facets = new LongValueFacetCounts("field", fc, false);
+    LongValueFacetCounts facets = new LongValueFacetCounts("field", fc);
 
     FacetResult result = facets.getAllChildrenSortByValue();
     assertEquals(
@@ -119,7 +119,7 @@ public class TestLongValueFacetCounts extends LuceneTestCase {
     IndexSearcher s = newSearcher(r);
     s.search(new MatchAllDocsQuery(), fc);
 
-    Facets facets = new LongValueFacetCounts("field", fc, false);
+    Facets facets = new LongValueFacetCounts("field", fc);
 
     List<FacetResult> result = facets.getAllDims(10);
     assertEquals(1, result.size());
@@ -199,7 +199,7 @@ public class TestLongValueFacetCounts extends LuceneTestCase {
       List<Map.Entry<Long, Integer>> expectedCounts = new ArrayList<>(expected.entrySet());
 
       // sort by value
-      Collections.sort(expectedCounts, (a, b) -> (Long.compare(a.getKey(), b.getKey())));
+      expectedCounts.sort(Comparator.comparingLong(Map.Entry::getKey));
 
       LongValueFacetCounts facetCounts;
       if (random().nextBoolean()) {
@@ -214,7 +214,7 @@ public class TestLongValueFacetCounts extends LuceneTestCase {
           if (VERBOSE) {
             System.out.println("  use doc values");
           }
-          facetCounts = new LongValueFacetCounts("field", fc, false);
+          facetCounts = new LongValueFacetCounts("field", fc);
         }
       } else {
         // optimized count all:
@@ -228,7 +228,7 @@ public class TestLongValueFacetCounts extends LuceneTestCase {
           if (VERBOSE) {
             System.out.println("  count all doc values");
           }
-          facetCounts = new LongValueFacetCounts("field", r, false);
+          facetCounts = new LongValueFacetCounts("field", r);
         }
       }
 
@@ -242,8 +242,7 @@ public class TestLongValueFacetCounts extends LuceneTestCase {
           Integer.MAX_VALUE);
 
       // sort by count
-      Collections.sort(
-          expectedCounts,
+      expectedCounts.sort(
           (a, b) -> {
             int cmp = -Integer.compare(a.getValue(), b.getValue());
             if (cmp == 0) {
@@ -288,7 +287,7 @@ public class TestLongValueFacetCounts extends LuceneTestCase {
         if (VERBOSE) {
           System.out.println("  use doc values");
         }
-        facetCounts = new LongValueFacetCounts("field", fc, false);
+        facetCounts = new LongValueFacetCounts("field", fc);
       } else {
         if (VERBOSE) {
           System.out.println("  use value source");
@@ -314,7 +313,7 @@ public class TestLongValueFacetCounts extends LuceneTestCase {
       expectedCounts = new ArrayList<>(expected.entrySet());
 
       // sort by value
-      Collections.sort(expectedCounts, (a, b) -> (Long.compare(a.getKey(), b.getKey())));
+      expectedCounts.sort(Comparator.comparingLong(Map.Entry::getKey));
       actual = facetCounts.getAllChildrenSortByValue();
       assertSame(
           "id " + minId + "-" + maxId + ", sort facets by value",
@@ -325,8 +324,7 @@ public class TestLongValueFacetCounts extends LuceneTestCase {
           Integer.MAX_VALUE);
 
       // sort by count
-      Collections.sort(
-          expectedCounts,
+      expectedCounts.sort(
           (a, b) -> {
             int cmp = -Integer.compare(a.getValue(), b.getValue());
             if (cmp == 0) {
@@ -447,7 +445,7 @@ public class TestLongValueFacetCounts extends LuceneTestCase {
       List<Map.Entry<Long, Integer>> expectedCounts = new ArrayList<>(expected.entrySet());
 
       // sort by value
-      Collections.sort(expectedCounts, (a, b) -> (Long.compare(a.getKey(), b.getKey())));
+      expectedCounts.sort(Comparator.comparingLong(Map.Entry::getKey));
 
       LongValueFacetCounts facetCounts;
       if (random().nextBoolean()) {
@@ -455,13 +453,13 @@ public class TestLongValueFacetCounts extends LuceneTestCase {
         if (VERBOSE) {
           System.out.println("  use doc values");
         }
-        facetCounts = new LongValueFacetCounts("field", fc, true);
+        facetCounts = new LongValueFacetCounts("field", fc);
       } else {
         // optimized count all:
         if (VERBOSE) {
           System.out.println("  count all doc values");
         }
-        facetCounts = new LongValueFacetCounts("field", r, true);
+        facetCounts = new LongValueFacetCounts("field", r);
       }
 
       FacetResult actual = facetCounts.getAllChildrenSortByValue();
@@ -474,8 +472,7 @@ public class TestLongValueFacetCounts extends LuceneTestCase {
           Integer.MAX_VALUE);
 
       // sort by count
-      Collections.sort(
-          expectedCounts,
+      expectedCounts.sort(
           (a, b) -> {
             int cmp = -Integer.compare(a.getValue(), b.getValue());
             if (cmp == 0) {
@@ -517,7 +514,7 @@ public class TestLongValueFacetCounts extends LuceneTestCase {
       fc = new FacetsCollector();
       s.search(IntPoint.newRangeQuery("id", minId, maxId), fc);
       // cannot use value source here because we are multi valued
-      facetCounts = new LongValueFacetCounts("field", fc, true);
+      facetCounts = new LongValueFacetCounts("field", fc);
 
       expected = new HashMap<>();
       expectedChildCount = 0;
@@ -538,7 +535,7 @@ public class TestLongValueFacetCounts extends LuceneTestCase {
       expectedCounts = new ArrayList<>(expected.entrySet());
 
       // sort by value
-      Collections.sort(expectedCounts, (a, b) -> (Long.compare(a.getKey(), b.getKey())));
+      expectedCounts.sort(Comparator.comparingLong(Map.Entry::getKey));
       actual = facetCounts.getAllChildrenSortByValue();
       assertSame(
           "id " + minId + "-" + maxId + ", sort facets by value",
@@ -549,8 +546,7 @@ public class TestLongValueFacetCounts extends LuceneTestCase {
           Integer.MAX_VALUE);
 
       // sort by count
-      Collections.sort(
-          expectedCounts,
+      expectedCounts.sort(
           (a, b) -> {
             int cmp = -Integer.compare(a.getValue(), b.getValue());
             if (cmp == 0) {