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/05/19 16:44:49 UTC

[GitHub] [lucene] gsmiller commented on a diff in pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

gsmiller commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r877264170


##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.hyperrectangle;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.LabelAndValue;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.search.DocIdSetIterator;
+
+/** Get counts given a list of HyperRectangles (which must be of the same type) */
+public class HyperRectangleFacetCounts extends Facets {
+  /** Hypper rectangles passed to constructor. */
+  protected final HyperRectangle[] hyperRectangles;
+
+  /** Counts, initialized in subclass. */
+  protected final int[] counts;
+
+  /** Our field name. */
+  protected final String field;
+
+  /** Number of dimensions for field */
+  protected final int dims;
+
+  /** Total number of hits. */
+  protected int totCount;
+
+  /**
+   * Create HyperRectangleFacetCounts using this
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, HyperRectangle... hyperRectangles) throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert areHyperRectangleDimsConsistent(hyperRectangles)
+        : "All hyper rectangles must be the same dimensionality";
+    this.field = field;
+    this.hyperRectangles = hyperRectangles;
+    this.dims = hyperRectangles[0].dims;
+    this.counts = new int[hyperRectangles.length];
+    count(field, hits.getMatchingDocs());
+  }
+
+  private boolean areHyperRectangleDimsConsistent(HyperRectangle[] hyperRectangles) {
+    int dims = hyperRectangles[0].dims;
+    return Arrays.stream(hyperRectangles).allMatch(hyperRectangle -> hyperRectangle.dims == dims);
+  }
+
+  /** Counts from the provided field. */
+  private void count(String field, List<FacetsCollector.MatchingDocs> matchingDocs)
+      throws IOException {
+
+    for (int i = 0; i < matchingDocs.size(); i++) {

Review Comment:
   minor: I'd suggest `for (FacetsCollector.MatchingDocs hits : matchingDocs)` as a slightly more idiomatic loop style since you don't actually care about the index.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.hyperrectangle;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.LabelAndValue;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.search.DocIdSetIterator;
+
+/** Get counts given a list of HyperRectangles (which must be of the same type) */
+public class HyperRectangleFacetCounts extends Facets {
+  /** Hypper rectangles passed to constructor. */
+  protected final HyperRectangle[] hyperRectangles;
+
+  /** Counts, initialized in subclass. */
+  protected final int[] counts;
+
+  /** Our field name. */
+  protected final String field;
+
+  /** Number of dimensions for field */
+  protected final int dims;
+
+  /** Total number of hits. */
+  protected int totCount;
+
+  /**
+   * Create HyperRectangleFacetCounts using this
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, HyperRectangle... hyperRectangles) throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert areHyperRectangleDimsConsistent(hyperRectangles)
+        : "All hyper rectangles must be the same dimensionality";
+    this.field = field;
+    this.hyperRectangles = hyperRectangles;
+    this.dims = hyperRectangles[0].dims;
+    this.counts = new int[hyperRectangles.length];
+    count(field, hits.getMatchingDocs());
+  }
+
+  private boolean areHyperRectangleDimsConsistent(HyperRectangle[] hyperRectangles) {
+    int dims = hyperRectangles[0].dims;
+    return Arrays.stream(hyperRectangles).allMatch(hyperRectangle -> hyperRectangle.dims == dims);
+  }
+
+  /** Counts from the provided field. */
+  private void count(String field, List<FacetsCollector.MatchingDocs> matchingDocs)
+      throws IOException {
+
+    for (int i = 0; i < matchingDocs.size(); i++) {
+
+      FacetsCollector.MatchingDocs hits = matchingDocs.get(i);
+
+      BinaryDocValues binaryDocValues = DocValues.getBinary(hits.context.reader(), field);
+
+      final DocIdSetIterator it = hits.bits.iterator();
+      if (it == null) {
+        continue;
+      }

Review Comment:
   I think it's a little simpler to read if you create your iterator like this:
   
   ```
   BinaryDocValues binaryDocValues = DocValues.getBinary(hits.context.reader(), field);
         final DocIdSetIterator it = 
           ConjunctionUtils.intersectIterators(Arrays.asList(hits.bits.iterator(), binaryDocValues));
   ```
   
   ... then you don't have to separately advance your doc values iterator (and check that it advanced to the doc) as the loop will take care of all that for you.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.hyperrectangle;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.LabelAndValue;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.search.DocIdSetIterator;
+
+/** Get counts given a list of HyperRectangles (which must be of the same type) */
+public class HyperRectangleFacetCounts extends Facets {
+  /** Hypper rectangles passed to constructor. */
+  protected final HyperRectangle[] hyperRectangles;
+
+  /** Counts, initialized in subclass. */
+  protected final int[] counts;
+
+  /** Our field name. */
+  protected final String field;
+
+  /** Number of dimensions for field */
+  protected final int dims;
+
+  /** Total number of hits. */
+  protected int totCount;
+
+  /**
+   * Create HyperRectangleFacetCounts using this
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, HyperRectangle... hyperRectangles) throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert areHyperRectangleDimsConsistent(hyperRectangles)
+        : "All hyper rectangles must be the same dimensionality";
+    this.field = field;
+    this.hyperRectangles = hyperRectangles;
+    this.dims = hyperRectangles[0].dims;
+    this.counts = new int[hyperRectangles.length];
+    count(field, hits.getMatchingDocs());
+  }
+
+  private boolean areHyperRectangleDimsConsistent(HyperRectangle[] hyperRectangles) {
+    int dims = hyperRectangles[0].dims;
+    return Arrays.stream(hyperRectangles).allMatch(hyperRectangle -> hyperRectangle.dims == dims);
+  }
+
+  /** Counts from the provided field. */
+  private void count(String field, List<FacetsCollector.MatchingDocs> matchingDocs)
+      throws IOException {
+
+    for (int i = 0; i < matchingDocs.size(); i++) {
+
+      FacetsCollector.MatchingDocs hits = matchingDocs.get(i);
+
+      BinaryDocValues binaryDocValues = DocValues.getBinary(hits.context.reader(), field);
+
+      final DocIdSetIterator it = hits.bits.iterator();
+      if (it == null) {
+        continue;
+      }
+
+      for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+        if (binaryDocValues.advanceExact(doc)) {
+          long[] point = LongPoint.unpack(binaryDocValues.binaryValue());
+          assert point.length == dims
+              : "Point dimension (dim="
+                  + point.length
+                  + ") is incompatible with hyper rectangle dimension (dim="
+                  + dims
+                  + ")";
+          // linear scan, change this to use R trees
+          boolean docIsValid = false;
+          for (int j = 0; j < hyperRectangles.length; j++) {
+            boolean validPoint = true;
+            for (int dim = 0; dim < dims; dim++) {
+              HyperRectangle.LongRangePair range = hyperRectangles[j].getComparableDimRange(dim);
+              if (!range.accept(point[dim])) {
+                validPoint = false;
+                break;
+              }
+            }

Review Comment:
   OK, I think you can make this just a little more readable and avoid a boolean flag here if you use a labeled loop like this:
   
   ```
             ranges:
             for (int j = 0; j < hyperRectangles.length; j++) {
               for (int dim = 0; dim < dims; dim++) {
                 HyperRectangle.LongRangePair range = hyperRectangles[j].getComparableDimRange(dim);
                 if (!range.accept(point[dim])) {
                   continue ranges;
                 }
               }
               counts[j]++;
               docIsValid = true;
             }
   ```



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.hyperrectangle;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.LabelAndValue;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.search.DocIdSetIterator;
+
+/** Get counts given a list of HyperRectangles (which must be of the same type) */
+public class HyperRectangleFacetCounts extends Facets {
+  /** Hypper rectangles passed to constructor. */
+  protected final HyperRectangle[] hyperRectangles;
+
+  /** Counts, initialized in subclass. */
+  protected final int[] counts;
+
+  /** Our field name. */
+  protected final String field;
+
+  /** Number of dimensions for field */
+  protected final int dims;
+
+  /** Total number of hits. */
+  protected int totCount;
+
+  /**
+   * Create HyperRectangleFacetCounts using this
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, HyperRectangle... hyperRectangles) throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert areHyperRectangleDimsConsistent(hyperRectangles)
+        : "All hyper rectangles must be the same dimensionality";
+    this.field = field;
+    this.hyperRectangles = hyperRectangles;
+    this.dims = hyperRectangles[0].dims;
+    this.counts = new int[hyperRectangles.length];
+    count(field, hits.getMatchingDocs());
+  }
+
+  private boolean areHyperRectangleDimsConsistent(HyperRectangle[] hyperRectangles) {
+    int dims = hyperRectangles[0].dims;
+    return Arrays.stream(hyperRectangles).allMatch(hyperRectangle -> hyperRectangle.dims == dims);
+  }
+
+  /** Counts from the provided field. */
+  private void count(String field, List<FacetsCollector.MatchingDocs> matchingDocs)
+      throws IOException {
+
+    for (int i = 0; i < matchingDocs.size(); i++) {
+
+      FacetsCollector.MatchingDocs hits = matchingDocs.get(i);
+
+      BinaryDocValues binaryDocValues = DocValues.getBinary(hits.context.reader(), field);
+
+      final DocIdSetIterator it = hits.bits.iterator();
+      if (it == null) {
+        continue;
+      }
+
+      for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+        if (binaryDocValues.advanceExact(doc)) {
+          long[] point = LongPoint.unpack(binaryDocValues.binaryValue());

Review Comment:
   I wonder if we can avoid unpacking every stored point and instead check directly against the packed format? `PointRangeQuery` actually does this nicely. Instead of unpacking each point, we could pack our hypterrectangle ranges and then compare arrays with a `ByteArrayComparator`. Maybe have a look at the `Weight` created in `PointRangeQuery#createWeight` and see if something similar would make sense here.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.hyperrectangle;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.LabelAndValue;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.search.DocIdSetIterator;
+
+/** Get counts given a list of HyperRectangles (which must be of the same type) */
+public class HyperRectangleFacetCounts extends Facets {
+  /** Hypper rectangles passed to constructor. */
+  protected final HyperRectangle[] hyperRectangles;
+
+  /** Counts, initialized in subclass. */
+  protected final int[] counts;
+
+  /** Our field name. */
+  protected final String field;
+
+  /** Number of dimensions for field */
+  protected final int dims;
+
+  /** Total number of hits. */
+  protected int totCount;
+
+  /**
+   * Create HyperRectangleFacetCounts using this
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, HyperRectangle... hyperRectangles) throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert areHyperRectangleDimsConsistent(hyperRectangles)
+        : "All hyper rectangles must be the same dimensionality";
+    this.field = field;
+    this.hyperRectangles = hyperRectangles;
+    this.dims = hyperRectangles[0].dims;
+    this.counts = new int[hyperRectangles.length];
+    count(field, hits.getMatchingDocs());
+  }
+
+  private boolean areHyperRectangleDimsConsistent(HyperRectangle[] hyperRectangles) {
+    int dims = hyperRectangles[0].dims;
+    return Arrays.stream(hyperRectangles).allMatch(hyperRectangle -> hyperRectangle.dims == dims);
+  }
+
+  /** Counts from the provided field. */
+  private void count(String field, List<FacetsCollector.MatchingDocs> matchingDocs)
+      throws IOException {
+
+    for (int i = 0; i < matchingDocs.size(); i++) {
+
+      FacetsCollector.MatchingDocs hits = matchingDocs.get(i);
+
+      BinaryDocValues binaryDocValues = DocValues.getBinary(hits.context.reader(), field);
+
+      final DocIdSetIterator it = hits.bits.iterator();
+      if (it == null) {
+        continue;
+      }
+
+      for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+        if (binaryDocValues.advanceExact(doc)) {
+          long[] point = LongPoint.unpack(binaryDocValues.binaryValue());

Review Comment:
   Also, does this imply that each document can only index a single point in this field? Can we support docs with multiple points?



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.hyperrectangle;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.LabelAndValue;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.search.DocIdSetIterator;
+
+/** Get counts given a list of HyperRectangles (which must be of the same type) */
+public class HyperRectangleFacetCounts extends Facets {
+  /** Hypper rectangles passed to constructor. */
+  protected final HyperRectangle[] hyperRectangles;
+
+  /** Counts, initialized in subclass. */
+  protected final int[] counts;
+
+  /** Our field name. */
+  protected final String field;
+
+  /** Number of dimensions for field */
+  protected final int dims;
+
+  /** Total number of hits. */
+  protected int totCount;
+
+  /**
+   * Create HyperRectangleFacetCounts using this
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, HyperRectangle... hyperRectangles) throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert areHyperRectangleDimsConsistent(hyperRectangles)
+        : "All hyper rectangles must be the same dimensionality";
+    this.field = field;
+    this.hyperRectangles = hyperRectangles;
+    this.dims = hyperRectangles[0].dims;
+    this.counts = new int[hyperRectangles.length];
+    count(field, hits.getMatchingDocs());
+  }
+
+  private boolean areHyperRectangleDimsConsistent(HyperRectangle[] hyperRectangles) {
+    int dims = hyperRectangles[0].dims;
+    return Arrays.stream(hyperRectangles).allMatch(hyperRectangle -> hyperRectangle.dims == dims);
+  }
+
+  /** Counts from the provided field. */
+  private void count(String field, List<FacetsCollector.MatchingDocs> matchingDocs)
+      throws IOException {
+
+    for (int i = 0; i < matchingDocs.size(); i++) {
+
+      FacetsCollector.MatchingDocs hits = matchingDocs.get(i);
+
+      BinaryDocValues binaryDocValues = DocValues.getBinary(hits.context.reader(), field);
+
+      final DocIdSetIterator it = hits.bits.iterator();
+      if (it == null) {
+        continue;
+      }
+
+      for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+        if (binaryDocValues.advanceExact(doc)) {
+          long[] point = LongPoint.unpack(binaryDocValues.binaryValue());
+          assert point.length == dims
+              : "Point dimension (dim="
+                  + point.length
+                  + ") is incompatible with hyper rectangle dimension (dim="
+                  + dims
+                  + ")";
+          // linear scan, change this to use R trees
+          boolean docIsValid = false;
+          for (int j = 0; j < hyperRectangles.length; j++) {
+            boolean validPoint = true;
+            for (int dim = 0; dim < dims; dim++) {
+              HyperRectangle.LongRangePair range = hyperRectangles[j].getComparableDimRange(dim);
+              if (!range.accept(point[dim])) {
+                validPoint = false;
+                break;
+              }
+            }
+            if (validPoint) {
+              counts[j]++;
+              docIsValid = true;
+            }
+          }
+          if (docIsValid) {
+            totCount++;
+          }
+        }
+      }
+    }
+  }
+
+  @Override
+  public FacetResult getTopChildren(int topN, String dim, String... path) throws IOException {

Review Comment:
   Could you add a note to the javadoc along with a TODO that references LUCENE-10550 please? This isn't really providing "top children" functionality, but actually providing "all children" functionality (same as range faceting). @Yuti-G is working on properly modeling "get all children" separately, so we should make sure this gets including in that work.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.hyperrectangle;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.LabelAndValue;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.search.DocIdSetIterator;
+
+/** Get counts given a list of HyperRectangles (which must be of the same type) */
+public class HyperRectangleFacetCounts extends Facets {
+  /** Hypper rectangles passed to constructor. */
+  protected final HyperRectangle[] hyperRectangles;
+
+  /** Counts, initialized in subclass. */
+  protected final int[] counts;
+
+  /** Our field name. */
+  protected final String field;
+
+  /** Number of dimensions for field */
+  protected final int dims;
+
+  /** Total number of hits. */
+  protected int totCount;
+
+  /**
+   * Create HyperRectangleFacetCounts using this
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, HyperRectangle... hyperRectangles) throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert areHyperRectangleDimsConsistent(hyperRectangles)
+        : "All hyper rectangles must be the same dimensionality";
+    this.field = field;
+    this.hyperRectangles = hyperRectangles;
+    this.dims = hyperRectangles[0].dims;
+    this.counts = new int[hyperRectangles.length];
+    count(field, hits.getMatchingDocs());
+  }
+
+  private boolean areHyperRectangleDimsConsistent(HyperRectangle[] hyperRectangles) {
+    int dims = hyperRectangles[0].dims;
+    return Arrays.stream(hyperRectangles).allMatch(hyperRectangle -> hyperRectangle.dims == dims);
+  }
+
+  /** Counts from the provided field. */
+  private void count(String field, List<FacetsCollector.MatchingDocs> matchingDocs)
+      throws IOException {
+
+    for (int i = 0; i < matchingDocs.size(); i++) {
+
+      FacetsCollector.MatchingDocs hits = matchingDocs.get(i);
+
+      BinaryDocValues binaryDocValues = DocValues.getBinary(hits.context.reader(), field);
+
+      final DocIdSetIterator it = hits.bits.iterator();
+      if (it == null) {
+        continue;
+      }
+
+      for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+        if (binaryDocValues.advanceExact(doc)) {
+          long[] point = LongPoint.unpack(binaryDocValues.binaryValue());
+          assert point.length == dims
+              : "Point dimension (dim="
+                  + point.length
+                  + ") is incompatible with hyper rectangle dimension (dim="
+                  + dims
+                  + ")";
+          // linear scan, change this to use R trees
+          boolean docIsValid = false;
+          for (int j = 0; j < hyperRectangles.length; j++) {
+            boolean validPoint = true;
+            for (int dim = 0; dim < dims; dim++) {
+              HyperRectangle.LongRangePair range = hyperRectangles[j].getComparableDimRange(dim);
+              if (!range.accept(point[dim])) {

Review Comment:
   minor: we tend to favor ` == false` instead of `!` in the codebase for readability and less likely hood of introducing a future bug



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,149 @@
+/*
+ * 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.hyperrectangle;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.LabelAndValue;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.search.DocIdSetIterator;
+
+/** Get counts given a list of HyperRectangles (which must be of the same type) */
+public class HyperRectangleFacetCounts extends Facets {
+  /** Hypper rectangles passed to constructor. */
+  protected final HyperRectangle[] hyperRectangles;
+
+  /** Counts, initialized in subclass. */
+  protected final int[] counts;
+
+  /** Our field name. */
+  protected final String field;
+
+  /** Number of dimensions for field */
+  protected final int dims;
+
+  /** Total number of hits. */
+  protected int totCount;
+
+  /**
+   * Create HyperRectangleFacetCounts using this
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, HyperRectangle... hyperRectangles) throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert areHyperRectangleDimsConsistent(hyperRectangles)
+        : "All hyper rectangles must be the same dimensionality";
+    this.field = field;
+    this.hyperRectangles = hyperRectangles;
+    this.dims = hyperRectangles[0].dims;
+    this.counts = new int[hyperRectangles.length];
+    count(field, hits.getMatchingDocs());
+  }
+
+  private boolean areHyperRectangleDimsConsistent(HyperRectangle[] hyperRectangles) {
+    int dims = hyperRectangles[0].dims;
+    return Arrays.stream(hyperRectangles).allMatch(hyperRectangle -> hyperRectangle.dims == dims);
+  }
+
+  /** Counts from the provided field. */
+  private void count(String field, List<FacetsCollector.MatchingDocs> matchingDocs)
+      throws IOException {
+
+    for (int i = 0; i < matchingDocs.size(); i++) {
+
+      FacetsCollector.MatchingDocs hits = matchingDocs.get(i);
+
+      BinaryDocValues binaryDocValues = DocValues.getBinary(hits.context.reader(), field);
+
+      final DocIdSetIterator it = hits.bits.iterator();
+      if (it == null) {
+        continue;
+      }
+
+      for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+        if (binaryDocValues.advanceExact(doc)) {
+          long[] point = LongPoint.unpack(binaryDocValues.binaryValue());
+          assert point.length == dims
+              : "Point dimension (dim="
+                  + point.length
+                  + ") is incompatible with hyper rectangle dimension (dim="
+                  + dims
+                  + ")";
+          // linear scan, change this to use R trees
+          boolean docIsValid = false;

Review Comment:
   `docIsValid` was a bit of a confusing name to me. This really captures the idea that the doc contributed to at least one HR right? Maybe something like `shouldCountDoc` or something? I dunno... naming is hard! :)



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