mikemccand commented on code in PR #13689:
URL: https://github.com/apache/lucene/pull/13689#discussion_r1745595485


##########
lucene/demo/src/java/org/apache/lucene/demo/facet/package-info.java:
##########
@@ -385,6 +385,12 @@
  * <p>Sampling support is implemented in {@link
  * org.apache.lucene.facet.RandomSamplingFacetsCollector}.
  *
+ * <h3 id="drf">Dynamic Range Facets</h3>

Review Comment:
   Thank you for updating package javadocs!



##########
lucene/CHANGES.txt:
##########
@@ -303,6 +303,9 @@ New Features
 
 * GITHUB#13678: Add support JDK 23 to the Panama Vectorization Provider. 
(Chris Hegarty)
 
+* GITHUB#13689: Dynamic range facets - create weighted ranges over numeric 
fields with counts per range.

Review Comment:
   Maybe make this a bit more verbose?  E.g. something like:
   
   ```
   Add a new faceting feature, dynamic range facets, which automatically picks 
a balanced set of numeric ranges based on the distribution of values that occur 
across all hits.  For use cases that have a highly variable numeric doc values 
field, such as "price" in an e-commerce application, this facet method is 
powerful as it allows the presented ranges to adapt depending on what hits the 
query actually matches.  This is in contrast to existing range faceting that 
requires the application to provide the specific fixed ranges up front.



##########
lucene/facet/src/java/org/apache/lucene/facet/range/DynamicRangeUtil.java:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.facet.range;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.InPlaceMergeSorter;
+
+/**
+ * Methods to create dynamic ranges for numeric fields.
+ *
+ * @lucene.experimental
+ */
+public final class DynamicRangeUtil {
+
+  private DynamicRangeUtil() {}
+
+  /**
+   * Construct dynamic ranges using the specified weight field to generate 
equi-weight range for the
+   * specified numeric bin field
+   *
+   * @param weightFieldName Name of the specified weight field
+   * @param weightValueSource Value source of the weight field
+   * @param fieldValueSource Value source of the value field
+   * @param facetsCollector FacetsCollector
+   * @param topN Number of requested ranges
+   * @param exec An executor service that is used to do the computation
+   * @return A list of DynamicRangeInfo that contains count, relevance, min, 
max, and centroid for
+   *     each range
+   */
+  public static List<DynamicRangeInfo> computeDynamicRanges(
+      String weightFieldName,
+      LongValuesSource weightValueSource,
+      LongValuesSource fieldValueSource,
+      FacetsCollector facetsCollector,
+      int topN,
+      ExecutorService exec)
+      throws IOException {
+
+    List<FacetsCollector.MatchingDocs> matchingDocsList = 
facetsCollector.getMatchingDocs();
+    int totalDoc = matchingDocsList.stream().mapToInt(matchingDoc -> 
matchingDoc.totalHits).sum();
+    long[] values = new long[totalDoc];
+    long[] weights = new long[totalDoc];
+    long totalWeight = 0;
+    int overallLength = 0;
+
+    List<Future<?>> futures = new ArrayList<>();
+    List<SegmentTask> tasks = new ArrayList<>();
+    for (FacetsCollector.MatchingDocs matchingDocs : matchingDocsList) {
+      if (matchingDocs.totalHits > 0) {
+        SegmentOutput segmentOutput = new 
SegmentOutput(matchingDocs.totalHits);
+
+        // [1] retrieve values and associated weights concurrently
+        SegmentTask task =
+            new SegmentTask(matchingDocs, fieldValueSource, weightValueSource, 
segmentOutput);
+        tasks.add(task);
+        futures.add(exec.submit(task));
+      }
+    }
+
+    // [2] wait for all segment runs to finish
+    for (Future<?> future : futures) {
+      try {
+        future.get();
+      } catch (InterruptedException ie) {
+        throw new RuntimeException(ie);
+      } catch (ExecutionException ee) {
+        IOUtils.rethrowAlways(ee.getCause());
+      }
+    }
+
+    // [3] merge the segment value and weight arrays into one array 
respectively and update the
+    // total weights
+    // and valid value length
+    for (SegmentTask task : tasks) {
+      SegmentOutput curSegmentOutput = task.segmentOutput;
+      // if segment total weight overflows, return null
+      if (curSegmentOutput == null) {
+        return null;
+      }
+
+      assert curSegmentOutput.values.length == curSegmentOutput.weights.length;
+
+      try {
+        totalWeight = Math.addExact(curSegmentOutput.segmentTotalWeight, 
totalWeight);
+      } catch (ArithmeticException ae) {
+        throw new IllegalArgumentException(
+            "weight field \"" + weightFieldName + "\": long totalWeight value 
out of bounds", ae);
+      }
+
+      int currSegmentLen = curSegmentOutput.segmentIdx;
+      System.arraycopy(curSegmentOutput.values, 0, values, overallLength, 
currSegmentLen);
+      System.arraycopy(curSegmentOutput.weights, 0, weights, overallLength, 
currSegmentLen);
+      overallLength += currSegmentLen;
+    }
+    return computeDynamicNumericRanges(values, weights, overallLength, 
totalWeight, topN);
+  }
+
+  private static class SegmentTask implements Callable<Void> {
+    private final FacetsCollector.MatchingDocs matchingDocs;
+    private final DocIdSetIterator matchingParentDocsItr;
+    private final LongValuesSource fieldValueSource;
+    private final LongValuesSource weightValueSource;
+    private SegmentOutput segmentOutput;
+
+    SegmentTask(
+        FacetsCollector.MatchingDocs matchingDocs,
+        LongValuesSource fieldValueSource,
+        LongValuesSource weightValueSource,
+        SegmentOutput segmentOutput)
+        throws IOException {
+      this.matchingDocs = matchingDocs;
+      this.matchingParentDocsItr = matchingDocs.bits.iterator();
+      this.fieldValueSource = fieldValueSource;
+      this.weightValueSource = weightValueSource;
+      this.segmentOutput = segmentOutput;
+    }
+
+    @Override
+    public Void call() throws Exception {
+      LongValues fieldValue = fieldValueSource.getValues(matchingDocs.context, 
null);
+      LongValues weightValue = 
weightValueSource.getValues(matchingDocs.context, null);
+      for (int doc = matchingParentDocsItr.nextDoc();
+          doc != DocIdSetIterator.NO_MORE_DOCS;
+          doc = matchingParentDocsItr.nextDoc()) {
+        // If this doc doesn't have a weight, we skip it.
+        if (fieldValue.advanceExact(doc) == false || 
weightValue.advanceExact(doc) == false) {
+          continue;
+        }
+
+        long curValue = fieldValue.longValue();
+
+        long curWeight = weightValue.longValue();
+        // We skip weights equal to zero, otherwise they can skew the ranges.
+        // Imagine all the weights were zero - any ranges would be valid.
+        if (curWeight == 0) {
+          continue;
+        }
+
+        segmentOutput.values[segmentOutput.segmentIdx] = curValue;
+        segmentOutput.weights[segmentOutput.segmentIdx] = curWeight;
+        try {
+          segmentOutput.segmentTotalWeight =
+              Math.addExact(segmentOutput.segmentTotalWeight, curWeight);
+        } catch (ArithmeticException ae) {
+          throw new IllegalArgumentException("segment long totalWeight value 
out of bounds", ae);
+        }
+        segmentOutput.segmentIdx++;
+      }
+      return null;
+    }
+  }
+
+  /** Holds field value array, weight array, totalWeight, valid value index 
for each segment */
+  private static final class SegmentOutput {
+    private final long[] values;
+    private final long[] weights;
+    private long segmentTotalWeight = 0;

Review Comment:
   No need for `= 0` -- it's java's default.



##########
lucene/demo/src/java/org/apache/lucene/demo/facet/DynamicRangeFacetsExample.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.demo.facet;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Locale;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.NumericDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsConfig;
+import org.apache.lucene.facet.range.DynamicRangeUtil;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LongValuesSource;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.store.ByteBuffersDirectory;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.NamedThreadFactory;
+
+/**
+ * Demo dynamic range faceting.
+ *
+ * <p>The results look like so: min: 63 max: 75 centroid: 69.000000 count: 2 
weight: 137 min: 79
+ * max: 96 centroid: 86.000000 count: 3 weight: 83
+ *
+ * <p>We've computed dynamic ranges over popularity weighted by number of 
books. We can read the
+ * results as so: There are 137 books written by authors in the 63 to 75 
popularity range.
+ *
+ * <p>How it works: We collect all the values (popularity) and their weights 
(book counts). We sort
+ * the values and find the approximate weight per range. In this case the 
total weight is 220 (total
+ * books by all authors) and we want 2 ranges, so we're aiming for 110 books 
in each range. We add
+ * Chesterton to the first range, since he is the least popular author. He's 
written a lot of books,
+ * the range's weight is 90. We add Tolstoy to the first range, since he is 
next in line of
+ * popularity. He's written another 47 books, which brings the total weight to 
137. We're over the
+ * 110 target weight, so we stop and add everyone left to the second range.
+ */
+public class DynamicRangeFacetsExample {

Review Comment:
   And thank you for adding an example of this cool new feature to `demo` 
package too!



##########
lucene/facet/src/java/org/apache/lucene/facet/range/DynamicRangeUtil.java:
##########
@@ -0,0 +1,276 @@
+/*
+ * 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.facet.range;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.InPlaceMergeSorter;
+
+/**
+ * Methods to create dynamic ranges for numeric fields.
+ *
+ * @lucene.experimental
+ */
+public final class DynamicRangeUtil {
+
+  private DynamicRangeUtil() {}
+
+  /**
+   * Construct dynamic ranges using the specified weight field to generate 
equi-weight range for the
+   * specified numeric bin field
+   *
+   * @param weightFieldName Name of the specified weight field
+   * @param weightValueSource Value source of the weight field
+   * @param fieldValueSource Value source of the value field
+   * @param facetsCollector FacetsCollector
+   * @param topN Number of requested ranges
+   * @param exec An executor service that is used to do the computation
+   * @return A list of DynamicRangeInfo that contains count, relevance, min, 
max, and centroid for
+   *     each range
+   */
+  public static List<DynamicRangeInfo> computeDynamicRanges(
+      String weightFieldName,
+      LongValuesSource weightValueSource,
+      LongValuesSource fieldValueSource,
+      FacetsCollector facetsCollector,
+      int topN,
+      ExecutorService exec)
+      throws IOException {
+
+    List<FacetsCollector.MatchingDocs> matchingDocsList = 
facetsCollector.getMatchingDocs();
+    int totalDoc = matchingDocsList.stream().mapToInt(matchingDoc -> 
matchingDoc.totalHits).sum();
+    long[] values = new long[totalDoc];
+    long[] weights = new long[totalDoc];
+    long totalWeight = 0;
+    int overallLength = 0;
+
+    List<Future<?>> futures = new ArrayList<>();
+    List<SegmentTask> tasks = new ArrayList<>();
+    for (FacetsCollector.MatchingDocs matchingDocs : matchingDocsList) {
+      if (matchingDocs.totalHits > 0) {
+        SegmentOutput segmentOutput = new 
SegmentOutput(matchingDocs.totalHits);
+
+        // [1] retrieve values and associated weights concurrently
+        SegmentTask task =
+            new SegmentTask(matchingDocs, fieldValueSource, weightValueSource, 
segmentOutput);
+        tasks.add(task);
+        futures.add(exec.submit(task));
+      }
+    }
+
+    // [2] wait for all segment runs to finish
+    for (Future<?> future : futures) {
+      try {
+        future.get();
+      } catch (InterruptedException ie) {
+        throw new RuntimeException(ie);
+      } catch (ExecutionException ee) {
+        IOUtils.rethrowAlways(ee.getCause());
+      }
+    }
+
+    // [3] merge the segment value and weight arrays into one array 
respectively and update the
+    // total weights
+    // and valid value length
+    for (SegmentTask task : tasks) {
+      SegmentOutput curSegmentOutput = task.segmentOutput;
+      // if segment total weight overflows, return null
+      if (curSegmentOutput == null) {
+        return null;
+      }
+
+      assert curSegmentOutput.values.length == curSegmentOutput.weights.length;
+
+      try {
+        totalWeight = Math.addExact(curSegmentOutput.segmentTotalWeight, 
totalWeight);
+      } catch (ArithmeticException ae) {
+        throw new IllegalArgumentException(
+            "weight field \"" + weightFieldName + "\": long totalWeight value 
out of bounds", ae);
+      }
+
+      int currSegmentLen = curSegmentOutput.segmentIdx;
+      System.arraycopy(curSegmentOutput.values, 0, values, overallLength, 
currSegmentLen);
+      System.arraycopy(curSegmentOutput.weights, 0, weights, overallLength, 
currSegmentLen);
+      overallLength += currSegmentLen;
+    }
+    return computeDynamicNumericRanges(values, weights, overallLength, 
totalWeight, topN);
+  }
+
+  private static class SegmentTask implements Callable<Void> {
+    private final FacetsCollector.MatchingDocs matchingDocs;
+    private final DocIdSetIterator matchingParentDocsItr;
+    private final LongValuesSource fieldValueSource;
+    private final LongValuesSource weightValueSource;
+    private SegmentOutput segmentOutput;
+
+    SegmentTask(
+        FacetsCollector.MatchingDocs matchingDocs,
+        LongValuesSource fieldValueSource,
+        LongValuesSource weightValueSource,
+        SegmentOutput segmentOutput)
+        throws IOException {
+      this.matchingDocs = matchingDocs;
+      this.matchingParentDocsItr = matchingDocs.bits.iterator();
+      this.fieldValueSource = fieldValueSource;
+      this.weightValueSource = weightValueSource;
+      this.segmentOutput = segmentOutput;
+    }
+
+    @Override
+    public Void call() throws Exception {
+      LongValues fieldValue = fieldValueSource.getValues(matchingDocs.context, 
null);
+      LongValues weightValue = 
weightValueSource.getValues(matchingDocs.context, null);
+      for (int doc = matchingParentDocsItr.nextDoc();
+          doc != DocIdSetIterator.NO_MORE_DOCS;
+          doc = matchingParentDocsItr.nextDoc()) {
+        // If this doc doesn't have a weight, we skip it.
+        if (fieldValue.advanceExact(doc) == false || 
weightValue.advanceExact(doc) == false) {
+          continue;
+        }
+
+        long curValue = fieldValue.longValue();
+
+        long curWeight = weightValue.longValue();
+        // We skip weights equal to zero, otherwise they can skew the ranges.
+        // Imagine all the weights were zero - any ranges would be valid.
+        if (curWeight == 0) {
+          continue;
+        }
+
+        segmentOutput.values[segmentOutput.segmentIdx] = curValue;
+        segmentOutput.weights[segmentOutput.segmentIdx] = curWeight;
+        try {
+          segmentOutput.segmentTotalWeight =
+              Math.addExact(segmentOutput.segmentTotalWeight, curWeight);
+        } catch (ArithmeticException ae) {
+          throw new IllegalArgumentException("segment long totalWeight value 
out of bounds", ae);
+        }
+        segmentOutput.segmentIdx++;
+      }
+      return null;
+    }
+  }
+
+  /** Holds field value array, weight array, totalWeight, valid value index 
for each segment */
+  private static final class SegmentOutput {
+    private final long[] values;
+    private final long[] weights;
+    private long segmentTotalWeight = 0;
+    private int segmentIdx = 0;
+
+    public SegmentOutput(int hitsLength) {
+      this.values = new long[hitsLength];
+      this.weights = new long[hitsLength];
+    }
+  }
+
+  /**
+   * Compute dynamic numeric ranges using weights.
+   *
+   * @param values an array that contains the values of matching documents
+   * @param weights an array that contains the weights of matching documents
+   * @param len actual length of values and weights
+   * @param totalWeight the sum of weight values
+   * @param topN the requested top-n parameter
+   * @return A list of DynamicRangeInfo that contains count, relevance, min, 
max, and centroid
+   *     values for each range. The size of dynamic ranges may not be exactly 
equal to top-N. top-N
+   *     is used to compute the equi-weight per bin.
+   */
+  public static List<DynamicRangeInfo> computeDynamicNumericRanges(
+      long[] values, long[] weights, int len, long totalWeight, int topN) {
+    assert values.length == weights.length && len <= values.length;
+    List<DynamicRangeInfo> dynamicRangeResult = new ArrayList<>();
+    if (len == 0) {
+      return dynamicRangeResult;
+    }
+

Review Comment:
   Maybe also check/require `topN > 0`?



-- 
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-unsubscr...@lucene.apache.org

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


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

Reply via email to