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


##########
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:
   You're absolutely right. I was looking at `totalWeight` in 
`computeDynamicRanges` 🤦 
   Fixing it now.
   
   Edit: Fixed 
[here](https://github.com/apache/lucene/commit/955986d836d208de4e513e91573c424e420b30e6).



-- 
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