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/06/13 16:36:53 UTC

[GitHub] [lucene] mdmarshmallow opened a new pull request, #841: LUCENE-10274: Add hyperrectangle faceting capabilities

mdmarshmallow opened a new pull request, #841:
URL: https://github.com/apache/lucene/pull/841

   <!--
   _(If you are a project committer then you may remove some/all of the following template.)_
   
   Before creating a pull request, please file an issue in the ASF Jira system for Lucene:
   
   * https://issues.apache.org/jira/projects/LUCENE
   
   You will need to create an account in Jira in order to create an issue.
   
   The title of the PR should reference the Jira issue number in the form:
   
   * LUCENE-####: <short description of problem or changes>
   
   LUCENE must be fully capitalized. A short description helps people scanning pull requests for items they can work on.
   
   Properly referencing the issue in the title ensures that Jira is correctly updated with code review comments and commits. -->
   
   
   # Description
   
   Added basic hyperrectangle faceting capabilities. This is mostly just a draft PR to sketch out what the API will look like. Added new fields to store points as a BinaryDocValues field and then just linearly scan through those points to see if they "fit" inside the hyperrectangle. There are several important things that are still missing in this commit:
   * Current implementation only supports single-values point fields
   * Can optimize this implementation (Current idea is allowing user to choose whether to put input hyperrectangle in R tree for faster checks than linear scanning, however this wouldn't really be useful below a certain number of input hyperrectangled depending on R-tree node size)
   * Need to add more comprehensive tests.
   
   # Solution
   
   Currently just linear scans stored points through provided hyper rectangles and checks if a doc is accepted or not.
   
   # Tests
   
   Created two basic tests, will need to add more once the API is more set in stone.
   
   # Checklist
   
   Please review the following and check all that apply:
   
   - [X] I have reviewed the guidelines for [How to Contribute](https://github.com/apache/lucene/blob/main/CONTRIBUTING.md) and my code conforms to the standards described there to the best of my ability.
   - [X] I have created a Jira issue and added the issue ID to my pull request title.
   - [X] I have given Lucene maintainers [access](https://help.github.com/en/articles/allowing-changes-to-a-pull-request-branch-created-from-a-fork) to contribute to my PR branch. (optional but recommended)
   - [X] I have developed this patch against the `main` branch.
   - [X] I have run `./gradlew check`.
   - [X] I have added tests for my changes.
   


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r879870847


##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangle.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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;
+
+/** Holds the name and the number of dims for a HyperRectangle */
+public abstract class HyperRectangle {

Review Comment:
   Does `HyperRectangle` itself actually need to be part of the public API though? Users certainly need the definitions for `Long/DoubleHyperRectangle` but do they need the `HyperRectangle` definition itself? Like would they need a generic reference to `HyperRectangle`? I'm not sure?



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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r879866385


##########
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:
   Yeah, this convenience is nice. It also might optimize a little internally by figuring out what to lead with, etc. for doing the conjunction. So definitely nice to use.



##########
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:
   Nice!



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


[GitHub] [lucene] mdmarshmallow commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1148091050

   Ok so I took a look at the additions you made to get more of an understanding of what is going on here. I'll try to explain my understanding, and let me know if there is anything wrong with what I'm saying. I think the biggest difference between our implementations is the naming scheme. So it seems like the `FacetSetMatcher` class is equivalent to a `HyperRectangle`. So we have an `ExactFacetSetMatcher`, which would be equivalent to making a HyperRectangle where all the min's and the max's are the same for every range (in other words, a point), and I think you proposed above a `RangeFacetSetMatcher`, which would just be a regular hyper rectangle (albeit under a different name).
   
   I wanted to address this though:
   
   > I hope that with this API we'll also pave the way for users to realize
   > they can implement their own `FacetSetMatcher`, for instance treating the
   > first 2 dimensions as range, and 3rd and 4th as exact (again, to create
   > specialized matchers).
   
   I think this is something that we should provide out of the box right? It seems like it could be common enough if someone is using this functionality. Maybe something like `RangedAndExactFacetSetMatcher` that lets user specify which which dimensions they want as exact matches, the details don't matter too much right now.
   
   Also for this point:
   
   >  I also think that the proposed API under `facetset` is easier to
   > extend, even though I'm sure we can re-structure the `hyperrectangle`
   > package to allow for such extension. Essentially you want a _Reader_ which
   > knows how to read the `long[]` and a `Filter/Matcher/Whatever` which
   > interprets them, returning a boolean or something else. That part is the
   > extension point we'd hope users to implement, which will make the
   > underlying storage of the points an abstraction that users don't have to
   > deal with.
   
   I get where your coming from, but I still feel like overriding the `matches` function is still a bit of an expert use case and no super clear maybe?
   
   I have a proposal here that may be even more convoluted and a bit crazy, but I'll just put it out there in case. So starting with `FacetSet`. What if we made that `abstract FacetSet<T>` and provide an `abstract long[] writeToLong(T... values)`. Then the users will be able to store any value by extending this class and overwriting `writeToLong`. For example `LongFacetSet extends FacetSet<Long>`.
   
   Now with the matchers, I think if we provide an `ExactFacetSetMatcher`, a `RangeFacetSetMatcher`, and a `RangeAndFacetSetMatcher`, I think that would provide for all users' facet matching needs. So instead of letting the user override `match()`, we can create an `abstract readToLong()` that would read the field that the users created and stored. So for example `LongExactFacetSetMatcher extends ExactFacetSetMatcher`. Or maybe we can figure out a way to combine all the types of matchers into one to make this more simple, but we would try to have a more friendly API than what `HyperRectangle` has currently. Let me know what you think of this.


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1151477547

   Thanks @shaie! I'll have a look at the code you pushed. Before doing so, I'm going to take the opportunity to respond to your and Marc's questions/comments above :)
   
   @mdmarshmallow:
   > When we decide to optimize this (right after this PR is merged ideally), we would let MatchingFacetSetsCount be able to take a look at the FSM's passed to it and then determine if it should put the FSM into an R tree, KD tree, or just linearly scan based on the min and max of eachFSM. I think this makes sense, but we also shouldn't discuss it too much here as I think this is for another PR. I think the point is we can optimize the facetsets package in it's current state. With that being said, I do plan on writing the KD and R tree optimizations as soon as this is merged so I am still for this remaining a long[] API.
   
   Right, that's how I'm thinking of this. It's actually not too different than what the existing range faceting implementation does. That implementation looks at the requested ranges and determines how to optimize. Specifically, if none of the ranges overlap, it does one thing, but if any ranges do overlap, it takes a different approach. The caller isn't aware of any of this (nor should they be). I see this as similar (although a different problem). The caller describes the "matches" they want to facet on and `MatchingFacetSetCount` figures out the most optimal way to do that.
   
   @shaie:
   > [...] why we discussed having a fastMatchQuery optimization too [...]
   +1, makes sense
   
   > I think it's fine if we'll leave these optimizations for later, and even if that will change the API between MFSC and FSM, it's not a big deal yet.
   +1
   
   > We certainly can add such API. For "exact" matches it will return min=max right? Only for range ones they will be different. Are you proposing to return a min[] and max[] arrays, one entry per dimension? Just to make sure I understood your proposal (it doesn't have to be two arrays, but you understand the question).
   Right, that's how I'm thinking of it.
   
   > Not intending to start a discussion on how to implement that, but just wanted to point out that fastMatchQuery is something we'll need anyway (I guess) for drilldown, therefore it might be worth to start with it first? And, I'd rather we have some baseline benchmark before we implement any optimization, preferably for several use cases, so that in the end can propose which impl to use. E.g. maybe if you pass 1-2 FSMs, we won't need to create R/KD trees (they might even perform worse)? Anyway let's leave that for later.
   +1, totally agree. After this initial implementation, I would propose we create some benchmark tasks that utilize it so we can have a reasonably principled approach to optimizing.
   
   


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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1156805329

   > Maybe one thing would be that we want to make it mandatory for a `FacetSet` to implement its own `decode()` function?
   
   I've been there :). The reason I went w/ the static method and functional interface `FacetSetDecoder` is so that you don't have to construct an instance of a `FacetSet` only to use its `decode()` method. Essentially we'd want an `IntFacetSetDecoder` and `LongFacetSetDecoder` as they address int/float and long/double respectively. I can factor the two methods out to their own `FSD` extension if you think it will be more clear?


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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1159392950

   > Also, should we add `double` and `float` methods to `FacetSetDecoder`?
   
   I thought I answered wrongly but I didn't. To clarify, we encode `double` as sortable longs, and therefore use `decodeLongs` works. Similarly for `float`. Therefore I don't think we should introduce those `decodeFloats/Doubles` for matching purposes.


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1159091616

   @shaie 
   > I struggled back-and-forth between introducing a public final long[] comparableLongs on the abstract FacetSet to the getComparableLongs() method.
   
   +1 to the approach you went with. We can always change it, but I like how you have it personally.
   
   @mdmarshmallow 
   > I think we should also have some subpackages [...]
   
   I generally disagree with this. I _used_ to like breaking down functional areas into packages for organization, but it limits your ability to make classes/methods pkg-visible in order to expose a clean API. I now greatly prefer flatter packages with very limited APIs exposed.
   
   As for the `RangeMatching` interface, I'm not sure we need it? I think it's easy enough for a user to construct `LongRange` instances to pass to `RangeFacetSetMatcher` using the static factory methods (`fromDoubles`, `fromFloats`, etc.). It feels overly complicated to introduce `FacetSetRange<...>` and then require the different `FacetSet` implementations to implement these methods to deal with inclusive/exclusive boundaries. My only suggestion here might be to rename `LongRange` to just `Range`. The API may make a little more sense that way if dealing with something other than longs.
   
   Also, should we add `double` and `float` methods to `FacetSetDecoder`?
   
   And finally, +1 to having demo code that shows how to mix-and-match types within a single point. That would be interesting to write so we make sure this thing is really fully fleshed out. I think it is though.


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


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

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r903051834


##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/RangeFacetSetMatcher.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.facetset;
+
+import java.util.Arrays;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * A {@link FacetSetMatcher} which considers a set as a match if all dimensions fall within the
+ * given corresponding range.
+ *
+ * @lucene.experimental
+ */
+public class RangeFacetSetMatcher extends FacetSetMatcher {
+
+  private final long[] lowerRanges;
+  private final long[] upperRanges;
+
+  /**
+   * Constructs an instance to match facet sets with dimensions that fall within the given ranges.
+   */
+  public RangeFacetSetMatcher(String label, DimRange... dimRanges) {
+    super(label, getDims(dimRanges));
+    this.lowerRanges = Arrays.stream(dimRanges).mapToLong(range -> range.min).toArray();
+    this.upperRanges = Arrays.stream(dimRanges).mapToLong(range -> range.max).toArray();
+  }
+
+  @Override
+  public boolean matches(long[] dimValues) {
+    assert dimValues.length == dims
+        : "Encoded dimensions (dims="
+            + dimValues.length
+            + ") is incompatible with range dimensions (dims="
+            + dims
+            + ")";
+
+    for (int i = 0; i < dimValues.length; i++) {
+      if (dimValues[i] < lowerRanges[i]) {
+        // Doc's value is too low in this dimension
+        return false;
+      }
+      if (dimValues[i] > upperRanges[i]) {
+        // Doc's value is too high in this dimension
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private static int getDims(DimRange... dimRanges) {
+    if (dimRanges == null || dimRanges.length == 0) {
+      throw new IllegalArgumentException("dimRanges cannot be null or empty");
+    }
+    return dimRanges.length;
+  }
+
+  /**
+   * Creates a {@link DimRange} for the given min and max long values. This method is also suitable
+   * for int values.
+   */
+  public static DimRange fromLongs(long min, boolean minInclusive, long max, boolean maxInclusive) {

Review Comment:
   Yeah I think it makes sense in that case to extract DimRange.



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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1156131175

   > Rather than do this, I think that it makes more sense for the impls to just have the ability of returning their own values as a `long[]` (something like `getComparableLongs()` that was in the original `hyperrectangle` package). That way, we won't need to run the reader on every `long[]` we read from the DV but instead can compare it directly with `FacetSet#getComparableLongs()`. I think this would be more efficient?
   
   I pushed a commit which showcases how I _think_ we can (1) support custom encoding by `FacetSet` implementations and a `FacetSetDecoder` which knows how to read them. The matching side still happens on comparable `long[]` however now the encoding can use less bytes for e.g. integer and floats dimensions.
   
   I also added a `testIntFacetSetMatching` to show how to use the `IntFacet` and corresponding decoder. I didn't implement a test for Double/Float yet, I want us to digest the API and agree that it's what we want. If we're OK w/ this proposal, then we should address all of @gsmiller comments and beef up the tests.


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1145696479

   Trying to catch up on this now. I've been traveling and it's been difficult to find time. Thanks for all your thoughts @shaie!
   
   I think I'm only half-following your thoughts on the different APIs necessary, and will probably need to look at what you've documented in more detail. But... as a half-baked response, I'm not convinced (yet?) that we need this level of complexity in the API. In my mind, what we're trying to build is a generalization of what is already supported in long/double-range faceting (e.g., `LongRangeFacetCounts`), where the user specifies all the ranges they want counts for, we count hits against those ranges, and support returning those counts through a couple APIs. Those faceting implementations allow ranges to be specified in a single dimension, and determine which ranges the document points (in one-dimensional space) fall in.
   
   So "hyperrectangle faceting"—in my original thinking at least—is just a generalization of this to multiple dimensions. The points associated with the documents are in n-dimensional space, and the user specifies the different "hyperrectangles" they want counts for by providing a [min, max] range in each dimension. For cases like the "automotive parts finder" example, it's perfectly valid for the "hyperrectangles" provided by the user to also be single points (where the min/max are equivalent values in each dimension). But it's also valid to mix-and-match, where some dimensions are single points and some are ranges (e.g., "all auto parts that fit 'Chevy' (single point) for the years 2000 - 2010 (range)).
   
   In the situation where a user wants to "fix some dimension" and count over others, it can still be described as a set of "hyperrectangles," but where the specified ranges on some of the dimensions happen to be the same across all of them.
   
   So I'm not quite sure if what you're suggesting in the API is just syntactic sugar on top of this idea, or if we're possibly talking about different things here? I'll try to dive into your suggestion more though and understand. I feel like I'm just missing something important and need to catch up on your thinking. Thanks again for sharing! I'll circle back in a few days when I've (hopefully) had some more time to spend on this :)


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


[GitHub] [lucene] mdmarshmallow commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1148250328

   > We could totally offer that out-of-the-box, but I prefer that we do so in a different PR. Not sure about the API details yet, and whether it can truly be a generic impl or not, but it's totally something we should think about.
   
   This sounds like a good idea, I agree
   
   > Indeed, implementing your own FacetSetMatcher is expert?
   
   Ok, if this is the case, then should we provide more out of the box `FacetSetMatcher` classes (like a `double` and a `long` implementation for starters).
   
   > Extending FSM is what you refer to here? This is indeed the more expert API, which I hope users won't have to extend, given the OOTB classes.
   
   Yes I am referring to extending FSM, but as I said we should probably supply more OOTB classes in this case?
   
   > For this reason I don't think FSM should make you read the bytes into longs, let the impl do that if it's more convenient for it.
   
   Ah I see your complaints here. Yeah my goal was trying to make this less of an "expert" API, but if we are going to treat overriding `matches()` as an expert API and provide more OOTB classes, that makes sense.
   
   But one thing I want to point out that I was also thinking is that if we are able to read all `bytes[]` into `longs[]`, we can do some stuff with R-trees in `MatchingFacetSetCounts` like put all the FSM into an R-tree for faster matching. If we make it so that the data in an FSM isn't guaranteed to be able to be turned into a long, we can't do this optimization in the future.
   
   > Do you see more than two impls here?
   
   I could, for example if they wanted to encode String ordinals here where the ordinals are long, they can write that conversion logic in `writeToLong(String... values)`. With that being said, as you mentioned earlier this enforces encoding in longs which we do want to avoid right?


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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1150940692

   > For the single point case (`ExactFSM`), this is crazy right? Even if there are a small number of provided `ExactFSM` instances we're matching against, doing a linear scan of all of them for every point is pretty dang inefficient. Especially so for a case where there are many provided hits with many indexed points for each.
   
   That's true, and hence why we discussed having a `fastMatchQuery` optimization too (which will skip over hits that don't eg. have any "Ford" or "2010" dimensions). That's an optimization we should (IMO) add in the (near) future, after this PR.
   
   > I personally think we should design with this optimization in mind, but I think we're close and I don't actually think the current proposal needs to really change to allow for future optimizations.
   
   I agree! I think it's fine if we'll leave these optimizations for later, and even if that will change the API between `MFSC` and `FSM`, it's not a big deal yet.
   
   > if we want to put these optimizations in place, could we not just add a method to `FSM` that exposes the min/max values for each dimension?
   
   We certainly can add such API. For "exact" matches it will return `min=max` right? Only for range ones they will be different. Are you proposing to return a `min[]` and `max[]` arrays, one entry per dimension? Just to make sure I understood your proposal (it doesn't have to be two arrays, but you understand the question).
   
   > we would let `MatchingFacetSetsCount` be able to take a look at the `FSM`'s passed to it and then determine if it should put the FSM into an R tree, KD tree, or just linearly scan based on the `min` and `max` of each`FSM`
   
   Not intending to start a discussion on how to implement that, but just wanted to point out that `fastMatchQuery` is something we'll need anyway (I guess) for drilldown, therefore it might be worth to start with it first? And, I'd rather we have some baseline benchmark before we implement any optimization, preferably for several use cases, so that in the end can propose which impl to use. E.g. maybe if you pass 1-2 `FSMs`, we won't need to create R/KD trees (they might even perform worse)? Anyway let's leave that for later.
   
   > With that being said, I do plan on writing the KD and R tree optimizations as soon as this is merged so I am still for this remaining a `long[]` API.
   
   I don't mind if we do that, but since it seems like a trivial change to make after (it doesn't affect the end-user API, only the internal protocol between `MFSC` and `FSM`), and since maybe `fastMatchQuery` will be good enough to speed up `FacetSet` aggregations, we may not need to have a `long[]` API eventually? Just saying I feel like it's not a decision we have to take now. We can also add a helper `toLong[]` method to begin with, so that `MFSC` can convert the bytes to longs, do whatever R/KD-Tree-iness it needs and then call `FSM`? I don't know if what I wrote even makes sense, but I feel like changing the API to fit the optimizations when we'll come to implement them will be much clearer since we'll know what we need.


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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1154701271

   I pushed some more cleanups and minor refactoring.


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


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

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r876142339


##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.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 by 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)

Review Comment:
   Ok sounds good to me, I'll just use the single constructor then.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert isHyperRectangleDimsConsistent(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 isHyperRectangleDimsConsistent(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; ) {
+        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);

Review Comment:
   Hmm I don't think I can actually since `HyperRectangleFacetCoutns` doesn't inherit from `HyperRectangle` in any way (or at least Intellij is showing a compilation error when I try this).



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.util.Arrays;
+import org.apache.lucene.util.NumericUtils;
+
+/** Stores a hyper rectangle as an array of DoubleRangePairs */
+public class DoubleHyperRectangle extends HyperRectangle {
+
+  /** Creates DoubleHyperRectangle */
+  public DoubleHyperRectangle(String label, DoubleRangePair... pairs) {
+    super(label, convertToLongRangePairArray(pairs));
+  }
+
+  private static LongRangePair[] convertToLongRangePairArray(DoubleRangePair... pairs) {
+    if (pairs == null || pairs.length == 0) {
+      throw new IllegalArgumentException("Pairs cannot be null or empty");
+    }
+    return Arrays.stream(pairs).map(DoubleRangePair::toLongRangePair).toArray(LongRangePair[]::new);
+  }
+
+  /** Defines a single range in a DoubleHyperRectangle */
+  public static class DoubleRangePair {
+    /** Inclusive min */
+    public final double min;
+
+    /** Inclusive max */
+    public final double max;
+
+    /**
+     * Creates a DoubleRangePair, very similar to the constructor of {@link
+     * org.apache.lucene.facet.range.DoubleRange}
+     *
+     * @param minIn Min value of pair
+     * @param minInclusive If minIn is inclusive
+     * @param maxIn Max value of pair
+     * @param maxInclusive If maxIn is inclusive
+     */
+    public DoubleRangePair(double minIn, boolean minInclusive, double maxIn, boolean maxInclusive) {
+      if (Double.isNaN(minIn) || Double.isNaN(maxIn)) {
+        throw new IllegalArgumentException(
+            "min and max cannot be NaN: min=" + minIn + ", max=" + maxIn);
+      }
+
+      if (!minInclusive) {
+        minIn = Math.nextUp(minIn);
+      }
+
+      if (!maxInclusive) {
+        // Why no Math.nextDown?

Review Comment:
   Ah thanks for checking that out :). Changed to `Math.nextDown()`



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.util.Arrays;
+import org.apache.lucene.util.NumericUtils;
+
+/** Stores a hyper rectangle as an array of DoubleRangePairs */
+public class DoubleHyperRectangle extends HyperRectangle {
+
+  /** Creates DoubleHyperRectangle */
+  public DoubleHyperRectangle(String label, DoubleRangePair... pairs) {
+    super(label, convertToLongRangePairArray(pairs));
+  }
+
+  private static LongRangePair[] convertToLongRangePairArray(DoubleRangePair... pairs) {

Review Comment:
   Changed to `convertToLongRangePairs`



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert isHyperRectangleDimsConsistent(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 isHyperRectangleDimsConsistent(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; ) {
+        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++;
+          }
+        }
+        doc = it.nextDoc();

Review Comment:
   I think you are correct here. Changed it.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangle.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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;
+
+/** Holds the name and the number of dims for a HyperRectangle */
+public abstract class HyperRectangle {
+  /** Label that identifies this range. */
+  public final String label;
+
+  /** How many dimensions this hyper rectangle has (IE: a regular rectangle would have dims=2) */
+  public final int dims;
+
+  /** All subclasses should store pairs as comparable longs */
+  protected final LongRangePair[] pairs;
+
+  /** Sole constructor. */
+  protected HyperRectangle(String label, LongRangePair... pairs) {
+    if (label == null) {
+      throw new IllegalArgumentException("label must not be null");
+    }
+    if (pairs == null || pairs.length == 0) {
+      throw new IllegalArgumentException("Pairs cannot be null or empty");
+    }
+    this.label = label;
+    this.dims = pairs.length;
+    this.pairs = pairs;
+  }
+
+  /**
+   * Returns comparable long range for a provided dim
+   *
+   * @param dim dimension of the request range
+   * @return The comparable long version of the requested range
+   */
+  public LongRangePair getComparableDimRange(int dim) {
+    return pairs[dim];
+  }
+
+  /** Defines a single range in a HyperRectangle */
+  public static class LongRangePair {
+    /** Inclusive min */
+    public final long min;
+
+    /** Inclusive max */
+    public final long max;
+
+    /**
+     * Creates a LongRangePair, very similar to the constructor of {@link
+     * org.apache.lucene.facet.range.LongRange}
+     *
+     * @param minIn Min value of pair
+     * @param minInclusive If minIn is inclusive
+     * @param maxIn Max value of pair
+     * @param maxInclusive If maxIn is inclusive
+     */
+    public LongRangePair(long minIn, boolean minInclusive, long maxIn, boolean maxInclusive) {
+      if (!minInclusive) {
+        if (minIn != Long.MAX_VALUE) {
+          minIn++;
+        } else {
+          throw new IllegalArgumentException("Invalid min input, min=" + minIn);
+        }
+      }
+
+      if (!maxInclusive) {
+        if (maxIn != Long.MIN_VALUE) {
+          maxIn--;
+        } else {
+          throw new IllegalArgumentException("Invalid max input, max=" + maxIn);
+        }
+      }
+
+      if (minIn > maxIn) {
+        throw new IllegalArgumentException("Minimum cannot be greater than maximum");

Review Comment:
   Added max and min values, also did the same thing in `DoubleHyperRectangle`



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert isHyperRectangleDimsConsistent(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 isHyperRectangleDimsConsistent(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; ) {
+        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++;
+          }
+        }
+        doc = it.nextDoc();
+      }
+    }
+  }
+
+  @Override
+  public FacetResult getTopChildren(int topN, String dim, String... path) throws IOException {
+    validateTopN(topN);
+    if (field.equals(dim) == false) {
+      throw new IllegalArgumentException(
+          "invalid dim \"" + dim + "\"; should be \"" + field + "\"");
+    }
+    if (path.length != 0) {

Review Comment:
   Changed to `if (path != null && path.length != 0)`. Not sure what exactly you mean by doing this elsewhere, but this function is a copy of [`RangeFacetCounts#getTopChildren`](https://github.com/apache/lucene/blob/a071180a806d1bb7ae11ae30a07e43e452bea810/lucene/facet/src/java/org/apache/lucene/facet/range/RangeFacetCounts.java#L220).



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert isHyperRectangleDimsConsistent(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 isHyperRectangleDimsConsistent(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; ) {
+        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++;
+          }
+        }
+        doc = it.nextDoc();
+      }
+    }
+  }
+
+  @Override
+  public FacetResult getTopChildren(int topN, String dim, String... path) throws IOException {
+    validateTopN(topN);
+    if (field.equals(dim) == false) {
+      throw new IllegalArgumentException(
+          "invalid dim \"" + dim + "\"; should be \"" + field + "\"");
+    }
+    if (path.length != 0) {
+      throw new IllegalArgumentException("path.length should be 0");
+    }
+    LabelAndValue[] labelValues = new LabelAndValue[counts.length];
+    for (int i = 0; i < counts.length; i++) {
+      labelValues[i] = new LabelAndValue(hyperRectangles[i].label, counts[i]);
+    }
+    return new FacetResult(dim, path, totCount, labelValues, labelValues.length);
+  }
+
+  @Override
+  public Number getSpecificValue(String dim, String... path) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public List<FacetResult> getAllDims(int topN) throws IOException {
+    validateTopN(topN);
+    return Collections.singletonList(getTopChildren(topN, field));

Review Comment:
   Field would be the `dim` here, not that `path`.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert isHyperRectangleDimsConsistent(hyperRectangles)

Review Comment:
   Changed to `are`, thanks for pointing that out it did sound a bit awkward :)



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert isHyperRectangleDimsConsistent(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 isHyperRectangleDimsConsistent(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; ) {
+        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++;
+          }
+        }
+        doc = it.nextDoc();
+      }
+    }
+  }
+
+  @Override
+  public FacetResult getTopChildren(int topN, String dim, String... path) throws IOException {
+    validateTopN(topN);
+    if (field.equals(dim) == false) {
+      throw new IllegalArgumentException(
+          "invalid dim \"" + dim + "\"; should be \"" + field + "\"");
+    }
+    if (path.length != 0) {
+      throw new IllegalArgumentException("path.length should be 0");

Review Comment:
   Ah I think you're getting this part backwards, this enforces 0 path components. You're right in the having `path` here does not make sense.



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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r897537151


##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSetsField.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.facetset;
+
+import java.util.Arrays;
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A {@link BinaryDocValuesField} which encodes a list of {@link FacetSet facet sets}. The encoding
+ * scheme consists of a packed {@code long[]} where the first value denotes the number of dimensions
+ * in all the sets, followed by each set's values.
+ *
+ * @lucene.experimental
+ */
+public class FacetSetsField extends BinaryDocValuesField {
+
+  /**
+   * Create a new FacetSets field.
+   *
+   * @param name field name
+   * @param facetSets the {@link FacetSet} to index in that field. All must have the same number of
+   *     dimensions
+   * @throws IllegalArgumentException if the field name is null or the given facet sets are invalid
+   */
+  public static FacetSetsField create(String name, FacetSet... facetSets) {
+    validateFacetSets(facetSets);
+
+    return new FacetSetsField(name, toPackedLongs(facetSets));
+  }
+
+  private FacetSetsField(String name, BytesRef value) {
+    super(name, value);
+  }
+
+  private static void validateFacetSets(FacetSet... facetSets) {
+    if (facetSets == null || facetSets.length == 0) {
+      throw new IllegalArgumentException("FacetSets cannot be null or empty!");
+    }
+
+    int dims = facetSets[0].values.length;
+    if (!Arrays.stream(facetSets).allMatch(facetSet -> facetSet.values.length == dims)) {

Review Comment:
   Wasn't aware of this preference in the code base, will change to `noneMatch`



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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1153077923

   > I feel like having 2 `matches` functions in this case would make the API unnecessarily complex
   
   As I wrote before "_After we decide whether to stick w/ the long[] or byte[] API we'll remove the unneeded variant._". We won't have 2 APIs in the end.
   
   > [...] and then benchmark to see if the byte-based approach is _actually_ more optimal
   
   We will need to benchmark this of course, but I thought about this a bit and I feel like `long[]` will perform better for these reasons:
   
   1. The `FSM` will store a `long[]` and will iterate it instead of `byte[]`. Less array accesses and since it happens **for every** facet set in every matching docs, I feel like it's going to be more efficient than re-iterating the `byte[]`
   2. Likewise, `MFSC` will convert the BDV values **once** to `long[]` and then all `FSMs` will be given that `long[]` to match.
   
   It just feels like more optimal, unless you have a single `FSM`, but I don't think that will be a common case. Anyway, let's benchmark it, but with the analysis above, I also agree we should actually start with the `long[]` API, and replace it with a `byte[]` one only if actually performs better.
   
   


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


[GitHub] [lucene] gsmiller closed pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
gsmiller closed pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities
URL: https://github.com/apache/lucene/pull/841


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1156708360

   @shaie I like this API approach to supporting other data types. Thanks! I wonder if we shouldn't make `FacetSet` abstract and create a `LongFacetSet` for the long-specific impl (and `DoubleFacetSet` could extend it). It feels a little weird to me that `intFacetSet` / `FloatFacetSet` contain `long[]` member fields that go unused. Thoughts?
   
   I'll leave a couple small comments on the updated implementation here shortly as well, but I'm +1 on this approach to supporting other data types. Thanks again!


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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r898279453


##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/FloatFacetSet.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.facetset;
+
+import java.util.stream.IntStream;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.util.NumericUtils;
+
+/** A {@link FacetSet} which encodes float dimension values. */
+public class FloatFacetSet extends FacetSet {
+
+  /** The raw dimension values of this facet set. */
+  public final float[] floatValues;
+
+  /** Constructs a new instance of a facet set which stores {@code float} dimension values. */
+  public FloatFacetSet(float... values) {
+    super(toLongValues());
+
+    this.floatValues = values;
+  }
+
+  @Override
+  public int packValues(byte[] buf, int start) {
+    for (int i = 0, offset = start; i < floatValues.length; i++, offset += Float.BYTES) {
+      FloatPoint.encodeDimension(floatValues[i], buf, offset);
+    }
+    return floatValues.length * Float.BYTES;
+  }
+
+  @Override
+  public int sizePackedBytes() {
+    return values.length * Float.BYTES;
+  }
+
+  // Unfortunately there's no FloatStream to convert this in one line.
+  private static long[] toLongValues(float... values) {
+    if (values == null || values.length == 0) {
+      throw new IllegalArgumentException("values must not be null or empty");
+    }
+    return IntStream.range(0, values.length).mapToLong(NumericUtils::floatToSortableInt).toArray();

Review Comment:
   Oh what a silly bug 👍 , great catch!



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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1148322035

   > Ok, if this is the case, then should we provide more out of the box `FacetSetMatcher` classes (like a `double` and a `long` implementation for starters)
   
   Yes absolutely! I wrote somewhere in the comments that I only provided `Long` variants to talk about the API itself, but totally we should include a `Double` variant too.
   
   > I was also thinking is that if we are able to read all `bytes[]` into `longs[]`, we can do some stuff with R-trees
   
   That's a good point! So let me clarify the impl and my thoughts: originally you implemented it by reading the `byte[]` into a `long[]` and @gsmiller commented that you can compare the bytes directly, so I only flowed with this proposal (I assume it's for efficiency). But if we don't think that comparing bytes is necessary more efficient than converting to longs, but it does limit us in terms of the API in the future and where we'd want to take it to, then by all means let's make the matching API long-based.
   
   > I could, for example if they wanted to encode String ordinals here where the ordinals are long, they can write that conversion logic in `writeToLong(String... values)`
   
   So they can already do that, by passing the ordinals to the `FacetSet` constructor. I even demonstrated that in the test. Do you see an issue w/ that?


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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1157279381

   > Also another thing I realized is that `RangeFacetSetMatcher` is a bit awkward now, as it actually is only specific to `long`'s
   
   It's actually not only specific to longs. It uses longs to compare to ranges, like the HyperRectangle did. I took a look at your change and I've implemented something which I think is much more simple:
   
   * Add to `RangeFacetSetMatcher` `fromLongs()` and `fromDoubles()` which return a `LongRange` which only has a `min/max` inclusive ranges.
   * I don't think that we need to add an interface, nor have code like this to define a range. It's too cumbersome IMO, requires all 4 parameters to "align" on the size of the array:
   
   ```
   new LongFacetSet(FORD_ORD, 2010),
                       new boolean[] {true, true},
                       new LongFacetSet(FORD_ORD, 2014),
                       new boolean[] {true, true})),
   ```
   
   I also added a `testDoubleRange()` to showcase how simple it looks. I am not sure how to push the commit though without overriding your changes?


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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1157282173

   I'll try to paste excerpts of the code here:
   
   ```
     public static LongRange fromLongs(
         long min, boolean minInclusive, long max, boolean maxInclusive) {
         // here comes the current constructor code of LongRange
     }
   
     public static LongRange fromDoubles(
         double min, boolean minInclusive, double max, boolean maxInclusive) {
         // Here comes the code that we used to have in DoubleRangePair
     }
   
     // The new sole constructor of LongRange
     public LongRange(long min, long max) {
       this.min = min
       this.max = max;
     }
   ```
   
   To use it in tests it looks like this:
   
   ```
               new RangeFacetSetMatcher(
                   "Ford [2010-2014]",
                   singleDouble(FORD_ORD), // This calls RangeFacetSetMatcher.fromDoubles(...)
                   doubleRange(2010.0, true, 2014.0, true)), // This calls RangeFacetSetMatcher.fromDoubles(...)
               new RangeFacetSetMatcher(
                   "Ford (2010-2014]",
                   singleDouble(FORD_ORD),
                   doubleRange(2010.0, false, 2014.0, true)),
   ```
   
   WDYT?


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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1157313525

   Actually there weren't many conflicts so pushed my commit, we can now compare the two options side-by-side.


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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r901119754


##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSetsField.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.facetset;
+
+import java.util.Arrays;
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A {@link BinaryDocValuesField} which encodes a list of {@link FacetSet facet sets}. The encoding
+ * scheme consists of a packed {@code long[]} where the first value denotes the number of dimensions
+ * in all the sets, followed by each set's values.
+ *
+ * @lucene.experimental
+ */
+public class FacetSetsField extends BinaryDocValuesField {
+
+  /**
+   * Create a new FacetSets field.
+   *
+   * @param name field name
+   * @param facetSets the {@link FacetSet} to index in that field. All must have the same number of

Review Comment:
   Done



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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1162137822

   OK, I think I understand the intention with `FSD` long/int decoding more, but I think it could be a little confusing in the API currently. If I was a user, I'd expect there to be four implementations that correspond with the four types being supported out-of-the-box (int/long/float/double). But this is _really_ about knowing the width of the encoded "sortable longs" in the doc value field. So, with my better understanding, 1) I think the current approach is reasonable, and I can't think of any better suggestion, but 2) maybe we could update the javadocs in `FSD` to make it a little more clear it's about decoding the stored bytes into *comparable longs*?


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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1159759023

   @gsmiller, @mdmarshmallow I pushed another commit which completes the FacetSets document and adds another check ensuring all `FacetSet` given to the `FacetSetField` are actually of the same type (no sense indexing different types under the same field). The PR feels ready to me, except `RangeMatching` which I'm waiting for @mdmarshmallow to comment about whether he agrees to remove it or not. After we agree on that, I suggest that we do a final round of review and wrap up this PR.


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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r901238006


##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSetsField.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.facetset;
+
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A {@link BinaryDocValuesField} which encodes a list of {@link FacetSet facet sets}. The encoding
+ * scheme consists of a packed {@code byte[]} where the first value denotes the number of dimensions
+ * in all the sets, followed by each set's values.
+ *
+ * @lucene.experimental
+ */
+public class FacetSetsField extends BinaryDocValuesField {
+
+  /**
+   * Create a new FacetSets field.
+   *
+   * @param name field name
+   * @param facetSets the {@link FacetSet facet sets} to index in that field. All must have the same
+   *     number of dimensions
+   * @throws IllegalArgumentException if the field name is null or the given facet sets are invalid
+   */
+  public static FacetSetsField create(String name, FacetSet... facetSets) {
+    if (facetSets == null || facetSets.length == 0) {
+      throw new IllegalArgumentException("FacetSets cannot be null or empty!");
+    }
+
+    return new FacetSetsField(name, toPackedValues(facetSets));
+  }
+
+  private FacetSetsField(String name, BytesRef value) {
+    super(name, value);
+  }
+
+  private static BytesRef toPackedValues(FacetSet... facetSets) {
+    int numDims = facetSets[0].dims;
+    Class<?> expectedClass = facetSets[0].getClass();
+    byte[] buf = new byte[Integer.BYTES + facetSets[0].sizePackedBytes() * facetSets.length];
+    IntPoint.encodeDimension(numDims, buf, 0);
+    int offset = Integer.BYTES;
+    for (FacetSet facetSet : facetSets) {
+      if (facetSet.dims != numDims) {
+        throw new IllegalArgumentException(
+            "All FacetSets must have the same number of dimensions. Expected "
+                + numDims
+                + " found "
+                + facetSet.dims);
+      }
+      // It doesn't make sense to index facet sets of different types in the same field
+      if (facetSet.getClass() != expectedClass) {

Review Comment:
   Not sure what will we generify? E.g. you and I explored `FacetSet<T>` before but it complicates things and not sure it will work w/ e.g. the `TemperatureReadingFacetSet` (and the like) which mix several dimension types. Another thing - I don't want to over-complicate the API for something that is at the end of the day just extra safety, I can't see why would someone try to index two different `FacetSet` types in the same field and expect it to work.



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


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

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r875205786


##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.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 by 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)

Review Comment:
   Nothing really, I just wanted to make all the `HyperRectangle`'s be of the same subclass, though we could also leave it up to the user to decide whether they want that or not, in which case I could just do `HyperRectangle...`



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangle.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+/** Holds the name and the number of dims for a HyperRectangle */
+public abstract class HyperRectangle {
+  /** Label that identifies this range. */
+  public final String label;
+
+  /** How many dimensions this hyper rectangle has (IE: a regular rectangle would have dims=2) */
+  public final int dims;
+
+  /** Sole constructor. */
+  protected HyperRectangle(String label, int dims) {
+    if (label == null) {
+      throw new IllegalArgumentException("label must not be null");
+    }
+    if (dims <= 0) {
+      throw new IllegalArgumentException("Dims must be greater than 0. Dims=" + dims);
+    }
+    this.label = label;
+    this.dims = dims;
+  }
+
+  /**
+   * Converts hyper rectangles ranges into a comparable long from whatever type it is in
+   *
+   * @param dim dimension of the request range
+   * @return The comparable long version of the requested range
+   */
+  public abstract LongHyperRectangle.LongRangePair getComparableDimRange(int dim);

Review Comment:
   Moved the `LongRangePair` to `HyperRectangle` and made concrete implementation of `getComparableDimRange`. I ended up making a few code changes as a result of this but I think it is cleaner now.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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 org.apache.lucene.util.NumericUtils;
+
+/** Stores a hyper rectangle as an array of DoubleRangePairs */
+public class DoubleHyperRectangle extends HyperRectangle {
+
+  /** Stores pair as LongRangePair */
+  private final LongHyperRectangle.LongRangePair[] pairs;
+
+  /** Created DoubleHyperRectangle */

Review Comment:
   Typo, thanks for catching!



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    this.field = field;
+    this.hyperRectangles = hyperRectangles;
+    this.dims = hyperRectangles[0].dims;
+    assert isHyperRectangleDimsConsistent()
+        : "All hyper rectangles must be the same dimensionality";
+    this.counts = new int[hyperRectangles.length];
+    count(field, hits.getMatchingDocs());
+  }
+
+  private boolean isHyperRectangleDimsConsistent() {
+    for (HyperRectangle hyperRectangle : hyperRectangles) {

Review Comment:
   Didn't realize `allMatch` was a thing, thanks for the suggestion!



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoublePointFacetField.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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 org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * Takes an array of doubles and converts them to sortable longs, then stores as a {@link
+ * BinaryDocValuesField}
+ */
+public class DoublePointFacetField extends BinaryDocValuesField {
+
+  /**
+   * Creates a new DoublePointFacetField, indexing the provided N-dimensional long point.
+   *
+   * @param name field name
+   * @param point double[] value
+   * @throws IllegalArgumentException if the field name or value is null.
+   */
+  public DoublePointFacetField(String name, double... point) {
+    super(name, LongPoint.pack(convertToSortableLongPoint(point)));
+  }
+
+  private static long[] convertToSortableLongPoint(double[] point) {
+    long[] ret = new long[point.length];

Review Comment:
   Agree it makes the code look a lot cleaner :)



##########
lucene/core/src/java/org/apache/lucene/document/LongPoint.java:
##########
@@ -117,6 +117,25 @@ public static BytesRef pack(long... point) {
     return new BytesRef(packed);
   }
 
+  /**
+   * Unpack a BytesRef into a long point
+   *
+   * @param bytesRef BytesRef Value
+   * @throws IllegalArgumentException the value is null
+   */
+  public static long[] unpack(BytesRef bytesRef) {

Review Comment:
   If I understand correctly, are you suggesting to decode each dimension, and then check each dim against all the hyperrectangles before moving on to the next dim to do the same thing? I think that makes tracking the counts harder as the counts are per hyperrectangle. Let me know if I am misunderstanding this though. For now I'll keep it the way it is.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    this.field = field;
+    this.hyperRectangles = hyperRectangles;
+    this.dims = hyperRectangles[0].dims;
+    assert isHyperRectangleDimsConsistent()

Review Comment:
   Changed in next revision.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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 org.apache.lucene.util.NumericUtils;
+
+/** Stores a hyper rectangle as an array of DoubleRangePairs */
+public class DoubleHyperRectangle extends HyperRectangle {
+
+  /** Stores pair as LongRangePair */
+  private final LongHyperRectangle.LongRangePair[] pairs;
+
+  /** Created DoubleHyperRectangle */
+  public DoubleHyperRectangle(String label, DoubleRangePair... pairs) {
+    super(label, checkPairsAndGetDim(pairs));
+    this.pairs = new LongHyperRectangle.LongRangePair[pairs.length];
+    for (int dim = 0; dim < pairs.length; dim++) {
+      long longMin = NumericUtils.doubleToSortableLong(pairs[dim].min);
+      long longMax = NumericUtils.doubleToSortableLong(pairs[dim].max);
+      this.pairs[dim] = new LongHyperRectangle.LongRangePair(longMin, true, longMax, true);

Review Comment:
   I think passing `true` here always is fine since `min` and `max` are always inclusive themselves, the boolean values just determine whether the params provided are inclusive or not, but if exclusive they are changed to become inclusive. That being said, I think introducing a `toLongRangePair` function makes sense and will make the code easier to read.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    this.field = field;
+    this.hyperRectangles = hyperRectangles;
+    this.dims = hyperRectangles[0].dims;
+    assert isHyperRectangleDimsConsistent()
+        : "All hyper rectangles must be the same dimensionality";
+    this.counts = new int[hyperRectangles.length];
+    count(field, hits.getMatchingDocs());
+  }
+
+  private boolean isHyperRectangleDimsConsistent() {
+    for (HyperRectangle hyperRectangle : hyperRectangles) {
+      if (hyperRectangle.dims != this.dims) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /** 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; ) {
+        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++) {
+              LongHyperRectangle.LongRangePair range =
+                  hyperRectangles[j].getComparableDimRange(dim);
+              if (!range.accept(point[dim])) {
+                validPoint = false;
+                break;
+              }
+            }
+            if (validPoint) {
+              counts[j]++;
+              docIsValid = true;
+            }
+          }
+          if (docIsValid) {
+            totCount++;
+          }
+        }
+        doc = it.nextDoc();
+      }
+    }
+  }
+
+  @Override
+  public FacetResult getTopChildren(int topN, String dim, String... path) throws IOException {
+    validateTopN(topN);
+    if (dim.equals(field) == false) {

Review Comment:
   Changed in next revision



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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r876315286


##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangle.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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;
+
+/** Holds the name and the number of dims for a HyperRectangle */

Review Comment:
   nit: s/name/label/



##########
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) */

Review Comment:
   nit: we don't actually enforce the "same type" part. Do we really want/care to enforce that?



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert isHyperRectangleDimsConsistent(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 isHyperRectangleDimsConsistent(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; ) {
+        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++;
+          }
+        }
+        doc = it.nextDoc();
+      }
+    }
+  }
+
+  @Override
+  public FacetResult getTopChildren(int topN, String dim, String... path) throws IOException {
+    validateTopN(topN);
+    if (field.equals(dim) == false) {
+      throw new IllegalArgumentException(
+          "invalid dim \"" + dim + "\"; should be \"" + field + "\"");
+    }
+    if (path.length != 0) {
+      throw new IllegalArgumentException("path.length should be 0");

Review Comment:
   You seem to have missed this comment. If you wish to keep the check (although I feel it's not necessary), the please fix the error message ("should NOT be 0" or "should be greater than 0").



##########
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";

Review Comment:
   Note that you might trip NPE here I think, if someone doesn't pass any rectangle



##########
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. */

Review Comment:
   `counts` is actually initialized in this class. I also think that as javadocs, it's not very helpful. Maybe something like "Holds the number of matching documents (contain at least one intersecting point) for each HyperRectangle"?



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoublePointFacetField.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.util.Arrays;
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * Takes an array of doubles and converts them to sortable longs, then stores as a {@link
+ * BinaryDocValuesField}
+ */
+public class DoublePointFacetField extends BinaryDocValuesField {
+
+  /**
+   * Creates a new DoublePointFacetField, indexing the provided N-dimensional long point.
+   *
+   * @param name field name
+   * @param point double[] value
+   * @throws IllegalArgumentException if the field name or value is null.

Review Comment:
   We actually don't check if `point` is null? Not sure if you intended to



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


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

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r881945764


##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/LongPointFacetField.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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 org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.LongPoint;
+
+/** Packs an array of longs into a {@link BinaryDocValuesField} */
+public class LongPointFacetField extends BinaryDocValuesField {

Review Comment:
   ~~I was wondering what your thoughts were on just using separate numeric fields rather than packing them. I think this would make the API "nicer" to be honest, but the big drawback would that we would need some hacky multivalued implementation. I can think of some ways to build some sort of UnsortedNumericDV on top of SortedNumericDV, but they would all be super hacky and have limitations and probably not worth implementing.~~
   
   Edit: Upon thinking about this further, my suggestion doesn't make sense when we have multi-valued fields



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


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

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r881945764


##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/LongPointFacetField.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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 org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.LongPoint;
+
+/** Packs an array of longs into a {@link BinaryDocValuesField} */
+public class LongPointFacetField extends BinaryDocValuesField {

Review Comment:
   I was wondering what your thoughts were on just using separate numeric fields rather than packing them. I think this would make the API "nicer" to be honest, but the big drawback would that we would need some hacky multivalued implementation. I can think of some ways to build some sort of UnsortedNumericDV on top of SortedNumericDV, but they would all be super hacky and have limitations and probably not worth implementing.
   
   Edit: Upon thinking about this further, my suggestion doesn't make sense when we have multi-valued fields



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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1149113591

   > look forward to seeing the "range" implementation sketched out
   
   I pushed a commit w/ `RangeFacetSetMatcher` which is basically very similar to `HyperRectangle`, but w/ some different names (i.e. `LongRange` instead of `LongPair`). I've also refactored the tests into 3 classes: a general test case for `MatchingFacetSetCounts` and two classes for `Exact/RangeFacetSetMatcher`. The tests are not exhaustive yet, and the impl still lacks a `Double` variant.
   
   Do we want to consider moving to a `long[]` based matching API now? To allow for future optimizations like R/KD-Trees? Or proceed w/ the `byte[]` version for now (to finalize this PR, and until the need arises)?


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r879869751


##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/LongPointFacetField.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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 org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.LongPoint;
+
+/** Packs an array of longs into a {@link BinaryDocValuesField} */
+public class LongPointFacetField extends BinaryDocValuesField {

Review Comment:
   Full transparency: Marc and I had a discussion about this offline so I wanted to circle back here with a suggestion I made to him so it's fully out in the open and we can carry a conversation forward with the community.
   
   While I initially suggested adding this as a sub-class of `BinaryRangeDocValuesField` (similar to what `LongRangeDocValuesField` does), I wonder if the right thing would be to actually formalize a new doc values format type. If we're building faceting, and potentially "slow range query" support on top of these, it seems like formalizing the format encoding might be the right thing to do. I'd be really curious what the community thinks of this though, and recommended that Marc start that discussion. I'm personally leaning towards formalizing the format, and maybe even having single-valued and multi-valued versions (analogous to `(Sorted)NumericDocValues`).



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


[GitHub] [lucene] mdmarshmallow commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1154629173

   I agree with Greg, we should not let benchmarking block releasing this. I pushed a commit to remove the `byte[]` matches API


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


[GitHub] [lucene] mdmarshmallow commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1156814974

   Yeah I think having their own FSD would make it clear. I think we should also have some subpackages (maybe a `decoders`, `matchers`, and `sets` (for `FacetSets`))?


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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r878796006


##########
lucene/core/src/java/org/apache/lucene/document/DoublePointDocValuesField.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.document;

Review Comment:
   I am not opposed to this change, but I find it a bit strange that we add a "general" Point DV support without any tests that exercise it, and the only usage of it is in the Facet module. Do we see a use case in the future for other DV usage? Like Sorting?
   
   Anyway I'm fine either way, just wanted to comment here that since it's `@lucene.experimental` we could have also left it in the facet package and then move here if a more general use case came up.



##########
lucene/facet/src/test/org/apache/lucene/facet/hyperrectangle/TestHyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,374 @@
+/*
+ * 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 org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoublePointDocValuesField;
+import org.apache.lucene.document.LongPointDocValuesField;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.FacetTestCase;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollectorManager;
+import org.apache.lucene.facet.LabelAndValue;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.tests.index.RandomIndexWriter;
+
+public class TestHyperRectangleFacetCounts extends FacetTestCase {
+
+  public void testBasicLong() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+
+    for (long l = 0; l < 100; l++) {
+      Document doc = new Document();
+      LongPointDocValuesField field = new LongPointDocValuesField("field", l, l + 1L, l + 2L);
+      doc.add(field);
+      w.addDocument(doc);
+    }
+
+    // Also add point with Long.MAX_VALUE
+    Document doc = new Document();
+    LongPointDocValuesField field =
+        new LongPointDocValuesField(
+            "field", Long.MAX_VALUE - 2L, Long.MAX_VALUE - 1L, Long.MAX_VALUE);
+    doc.add(field);
+    w.addDocument(doc);
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    FacetsCollector fc = s.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    Facets facets =
+        new HyperRectangleFacetCounts(
+            "field",
+            fc,
+            new LongHyperRectangle(
+                "less than (10, 11, 12)",
+                new HyperRectangle.LongRangePair(0L, true, 10L, false),
+                new HyperRectangle.LongRangePair(0L, true, 11L, false),
+                new HyperRectangle.LongRangePair(0L, true, 12L, false)),
+            new LongHyperRectangle(
+                "less than or equal to (10, 11, 12)",
+                new HyperRectangle.LongRangePair(0L, true, 10L, true),
+                new HyperRectangle.LongRangePair(0L, true, 11L, true),
+                new HyperRectangle.LongRangePair(0L, true, 12L, true)),
+            new LongHyperRectangle(
+                "over (90, 91, 92)",

Review Comment:
   super nit: "Between (90,91,92) and (100,101,102)"? Cause two tests below we have "Over (1000...) which is really just Over, without a real upper limit. But feel free to ignore my pickiness :)



##########
lucene/facet/src/test/org/apache/lucene/facet/hyperrectangle/TestHyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,374 @@
+/*
+ * 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 org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoublePointDocValuesField;
+import org.apache.lucene.document.LongPointDocValuesField;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.FacetTestCase;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollectorManager;
+import org.apache.lucene.facet.LabelAndValue;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.tests.index.RandomIndexWriter;
+
+public class TestHyperRectangleFacetCounts extends FacetTestCase {
+
+  public void testBasicLong() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+
+    for (long l = 0; l < 100; l++) {
+      Document doc = new Document();
+      LongPointDocValuesField field = new LongPointDocValuesField("field", l, l + 1L, l + 2L);
+      doc.add(field);
+      w.addDocument(doc);
+    }
+
+    // Also add point with Long.MAX_VALUE
+    Document doc = new Document();
+    LongPointDocValuesField field =
+        new LongPointDocValuesField(
+            "field", Long.MAX_VALUE - 2L, Long.MAX_VALUE - 1L, Long.MAX_VALUE);
+    doc.add(field);
+    w.addDocument(doc);
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    FacetsCollector fc = s.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    Facets facets =
+        new HyperRectangleFacetCounts(
+            "field",
+            fc,
+            new LongHyperRectangle(
+                "less than (10, 11, 12)",
+                new HyperRectangle.LongRangePair(0L, true, 10L, false),
+                new HyperRectangle.LongRangePair(0L, true, 11L, false),
+                new HyperRectangle.LongRangePair(0L, true, 12L, false)),
+            new LongHyperRectangle(
+                "less than or equal to (10, 11, 12)",
+                new HyperRectangle.LongRangePair(0L, true, 10L, true),
+                new HyperRectangle.LongRangePair(0L, true, 11L, true),
+                new HyperRectangle.LongRangePair(0L, true, 12L, true)),
+            new LongHyperRectangle(
+                "over (90, 91, 92)",
+                new HyperRectangle.LongRangePair(90L, false, 100L, false),
+                new HyperRectangle.LongRangePair(91L, false, 101L, false),
+                new HyperRectangle.LongRangePair(92L, false, 102L, false)),
+            new LongHyperRectangle(
+                "(90, 91, 92) or above",

Review Comment:
   If you accept what I wrote above, then please change this too (and the double tests).



##########
lucene/core/src/java/org/apache/lucene/document/LongPointDocValuesField.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.document;
+
+/**
+ * Packs an array of longs into a {@link BinaryDocValuesField}

Review Comment:
   Can we make this jdoc consistent with the Double variant, mentioning that we're indexing Point values?



##########
lucene/core/src/java/org/apache/lucene/document/LongPointDocValuesField.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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.document;
+
+/**
+ * Packs an array of longs into a {@link BinaryDocValuesField}
+ *
+ * @lucene.experimental
+ */
+public class LongPointDocValuesField extends BinaryDocValuesField {
+
+  /**
+   * Creates a new LongPointFacetField, indexing the provided N-dimensional long point.

Review Comment:
   Same comment about `FacetField`



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangle.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.util.Arrays;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.util.ArrayUtil;
+
+/**
+ * Holds the label, the number of dims, and the point pairs for a HyperRectangle
+ *
+ * @lucene.experimental
+ */
+public abstract class HyperRectangle {
+  /** Label that identifies this range. */
+  public final String label;
+
+  /** How many dimensions this hyper rectangle has (IE: a regular rectangle would have dims=2) */
+  public final int dims;
+
+  private final ArrayUtil.ByteArrayComparator byteComparator =
+      ArrayUtil.getUnsignedComparator(Long.BYTES);
+
+  private final byte[] lowerPoints;
+  private final byte[] upperPoints;
+
+  /** Sole constructor. */
+  protected HyperRectangle(String label, LongRangePair... pairs) {
+    if (label == null) {
+      throw new IllegalArgumentException("label must not be null");
+    }
+    if (pairs == null || pairs.length == 0) {
+      throw new IllegalArgumentException("Pairs cannot be null or empty");
+    }
+    this.label = label;
+    this.dims = pairs.length;
+
+    this.lowerPoints =
+        LongPoint.pack(Arrays.stream(pairs).mapToLong(pair -> pair.min).toArray()).bytes;
+    this.upperPoints =
+        LongPoint.pack(Arrays.stream(pairs).mapToLong(pair -> pair.max).toArray()).bytes;
+  }
+
+  /**
+   * Checked a long packed value against this HyperRectangle. If you indexed a field with {@link
+   * org.apache.lucene.document.LongPointDocValuesField} or {@link
+   * org.apache.lucene.document.DoublePointDocValuesField}, those field values will be able to be

Review Comment:
   s/will be able to/can/?



##########
lucene/core/src/java/org/apache/lucene/document/DoublePointDocValuesField.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.document;
+
+import java.util.Arrays;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * Takes an array of doubles and converts them to sortable longs, then stores as a {@link
+ * BinaryDocValuesField}
+ *
+ * @lucene.experimental
+ */
+public class DoublePointDocValuesField extends BinaryDocValuesField {
+
+  /**
+   * Creates a new DoublePointFacetField, indexing the provided N-dimensional long point.

Review Comment:
   s/DoublePointFacetField/DoublePointDocValuesField/
   s/long point/double point/



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangle.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.util.Arrays;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.util.ArrayUtil;
+
+/**
+ * Holds the label, the number of dims, and the point pairs for a HyperRectangle
+ *
+ * @lucene.experimental
+ */
+public abstract class HyperRectangle {
+  /** Label that identifies this range. */
+  public final String label;
+
+  /** How many dimensions this hyper rectangle has (IE: a regular rectangle would have dims=2) */
+  public final int dims;
+
+  private final ArrayUtil.ByteArrayComparator byteComparator =
+      ArrayUtil.getUnsignedComparator(Long.BYTES);
+
+  private final byte[] lowerPoints;
+  private final byte[] upperPoints;
+
+  /** Sole constructor. */
+  protected HyperRectangle(String label, LongRangePair... pairs) {
+    if (label == null) {
+      throw new IllegalArgumentException("label must not be null");
+    }
+    if (pairs == null || pairs.length == 0) {
+      throw new IllegalArgumentException("Pairs cannot be null or empty");
+    }
+    this.label = label;
+    this.dims = pairs.length;
+
+    this.lowerPoints =
+        LongPoint.pack(Arrays.stream(pairs).mapToLong(pair -> pair.min).toArray()).bytes;
+    this.upperPoints =
+        LongPoint.pack(Arrays.stream(pairs).mapToLong(pair -> pair.max).toArray()).bytes;
+  }
+
+  /**
+   * Checked a long packed value against this HyperRectangle. If you indexed a field with {@link
+   * org.apache.lucene.document.LongPointDocValuesField} or {@link
+   * org.apache.lucene.document.DoublePointDocValuesField}, those field values will be able to be
+   * passed directly into this method.
+   *
+   * @param packedValue a byte array representing a long value
+   * @return whether the packed long point intersects with this HyperRectangle
+   */
+  public final boolean matches(byte[] packedValue) {
+    assert packedValue.length / Long.BYTES == dims
+        : "Point dimension (dim="
+            + packedValue.length / Long.BYTES
+            + ") is incompatible with hyper rectangle dimension (dim="
+            + dims
+            + ")";
+    for (int dim = 0; dim < dims; dim++) {

Review Comment:
   Instead of iterating on `dim` you can iterate on `offset` starting from 0 to `packedValue.length` and increment by `Long.BYTES`?



##########
lucene/core/src/java/org/apache/lucene/document/DoublePointDocValuesField.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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.document;
+
+import java.util.Arrays;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * Takes an array of doubles and converts them to sortable longs, then stores as a {@link
+ * BinaryDocValuesField}

Review Comment:
   nit: Maybe `A {@link BinaryDocValuesField} which indexes double point values as sortable-longs`?



##########
lucene/facet/src/test/org/apache/lucene/facet/hyperrectangle/TestHyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,374 @@
+/*
+ * 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 org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoublePointDocValuesField;
+import org.apache.lucene.document.LongPointDocValuesField;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.FacetTestCase;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollectorManager;
+import org.apache.lucene.facet.LabelAndValue;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.tests.index.RandomIndexWriter;
+
+public class TestHyperRectangleFacetCounts extends FacetTestCase {

Review Comment:
   Two questions about the tests:
   
   1. Would you like to add a test which verifies we assert that all given rectangles have the same dim?
   2. Would you like to add a test which showcases mixed Long/Double rectangles?



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


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

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r874247714


##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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 org.apache.lucene.util.NumericUtils;
+
+/** Stores a hyper rectangle as an array of DoubleRangePairs */
+public class DoubleHyperRectangle extends HyperRectangle {
+
+  /** Stores pair as LongRangePair */

Review Comment:
   Yes, thanks for catching that!



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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 org.apache.lucene.util.NumericUtils;
+
+/** Stores a hyper rectangle as an array of DoubleRangePairs */
+public class DoubleHyperRectangle extends HyperRectangle {
+
+  /** Stores pair as LongRangePair */
+  private final DoubleRangePair[] pairs;
+
+  /** Created DoubleHyperRectangle */
+  public DoubleHyperRectangle(String label, DoubleRangePair... pairs) {
+    super(label, pairs.length);

Review Comment:
   `pairs` should not be null or empty, added a check for both of these cases.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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 org.apache.lucene.util.NumericUtils;
+
+/** Stores a hyper rectangle as an array of DoubleRangePairs */
+public class DoubleHyperRectangle extends HyperRectangle {
+
+  /** Stores pair as LongRangePair */
+  private final DoubleRangePair[] pairs;
+
+  /** Created DoubleHyperRectangle */
+  public DoubleHyperRectangle(String label, DoubleRangePair... pairs) {
+    super(label, pairs.length);
+    this.pairs = pairs;
+  }
+
+  @Override
+  public LongHyperRectangle.LongRangePair getComparableDimRange(int dim) {
+    long longMin = NumericUtils.doubleToSortableLong(pairs[dim].min);
+    long longMax = NumericUtils.doubleToSortableLong(pairs[dim].max);
+    return new LongHyperRectangle.LongRangePair(longMin, true, longMax, true);
+  }
+
+  /** Defines a single range in a DoubleHyperRectangle */
+  public static class DoubleRangePair {
+    /** Inclusive min */
+    public final double min;
+
+    /** Inclusive max */
+    public final double max;
+
+    /**
+     * Creates a LongRangePair, very similar to the constructor of {@link
+     * org.apache.lucene.facet.range.DoubleRange}
+     *
+     * @param minIn Min value of pair
+     * @param minInclusive If minIn is inclusive
+     * @param maxIn Max value of pair
+     * @param maxInclusive If maxIn is inclusive
+     */
+    public DoubleRangePair(double minIn, boolean minInclusive, double maxIn, boolean maxInclusive) {
+      if (Double.isNaN(minIn)) {
+        throw new IllegalArgumentException("min cannot be NaN");
+      }
+      if (!minInclusive) {
+        minIn = Math.nextUp(minIn);
+      }
+
+      if (Double.isNaN(maxIn)) {

Review Comment:
   Thank you, will change this.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.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 by subclass. */

Review Comment:
   Oops typo, thanks for pointing that out!



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.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 by 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    this.field = field;
+    this.hyperRectangles = hyperRectangles;
+    this.dims = hyperRectangles[0].dims;
+    for (HyperRectangle hyperRectangle : hyperRectangles) {

Review Comment:
   Thanks for pointing that out, changed it.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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 org.apache.lucene.util.NumericUtils;
+
+/** Stores a hyper rectangle as an array of DoubleRangePairs */
+public class DoubleHyperRectangle extends HyperRectangle {
+
+  /** Stores pair as LongRangePair */
+  private final DoubleRangePair[] pairs;
+
+  /** Created DoubleHyperRectangle */
+  public DoubleHyperRectangle(String label, DoubleRangePair... pairs) {
+    super(label, pairs.length);
+    this.pairs = pairs;
+  }
+
+  @Override
+  public LongHyperRectangle.LongRangePair getComparableDimRange(int dim) {

Review Comment:
   Hmm I agree with your point that this method is unnecessarily expensive but instead of passing in a `LongPair`, I like the idea of just transforming all the inputs to `LongRangePairs` instead so this cost only gets incurred once per pair in the constructor rather than every time this method is called.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangle.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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;
+
+/** Holds the name and the number of dims for a HyperRectangle */
+public abstract class HyperRectangle {
+  /** Label that identifies this range. */
+  public final String label;
+
+  /** How many dimensions this hyper rectangle has (IE: a regular rectangle would have dims=2) */
+  public final int dims;
+
+  /** Sole constructor. */
+  protected HyperRectangle(String label, int dims) {
+    if (label == null) {
+      throw new NullPointerException("label must not be null");

Review Comment:
   I think this makes sense to be IAE. Changed it.



##########
lucene/facet/src/test/org/apache/lucene/facet/hyperrectangle/TestHyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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 org.apache.lucene.document.Document;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.FacetTestCase;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollectorManager;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.tests.index.RandomIndexWriter;
+
+public class TestHyperRectangleFacetCounts extends FacetTestCase {
+
+  public void testBasicLong() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+
+    for (long l = 0; l < 100; l++) {
+      Document doc = new Document();
+      LongPointFacetField field = new LongPointFacetField("field", l, l + 1L, l + 2L);
+      doc.add(field);
+      w.addDocument(doc);
+    }
+
+    // Also add point with Long.MAX_VALUE
+    Document doc = new Document();
+    LongPointFacetField field =
+        new LongPointFacetField("field", Long.MAX_VALUE - 2L, Long.MAX_VALUE - 1L, Long.MAX_VALUE);
+    doc.add(field);
+    w.addDocument(doc);
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    FacetsCollector fc = s.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    Facets facets =
+        new HyperRectangleFacetCounts(
+            "field",
+            fc,
+            new LongHyperRectangle(
+                "less than (10, 11, 12)",
+                new LongHyperRectangle.LongRangePair(0L, true, 10L, false),
+                new LongHyperRectangle.LongRangePair(0L, true, 11L, false),
+                new LongHyperRectangle.LongRangePair(0L, true, 12L, false)),
+            new LongHyperRectangle(
+                "less than or equal to (10, 11, 12)",
+                new LongHyperRectangle.LongRangePair(0L, true, 10L, true),
+                new LongHyperRectangle.LongRangePair(0L, true, 11L, true),
+                new LongHyperRectangle.LongRangePair(0L, true, 12L, true)),
+            new LongHyperRectangle(
+                "over (90, 91, 92)",
+                new LongHyperRectangle.LongRangePair(90L, false, 100L, false),
+                new LongHyperRectangle.LongRangePair(91L, false, 101L, false),
+                new LongHyperRectangle.LongRangePair(92L, false, 102L, false)),
+            new LongHyperRectangle(
+                "(90, 91, 92) or above",
+                new LongHyperRectangle.LongRangePair(90L, true, 100L, false),
+                new LongHyperRectangle.LongRangePair(91L, true, 101L, false),
+                new LongHyperRectangle.LongRangePair(92L, true, 102L, false)),
+            new LongHyperRectangle(
+                "over (1000, 1000, 1000)",
+                new LongHyperRectangle.LongRangePair(1000L, false, Long.MAX_VALUE - 2L, true),
+                new LongHyperRectangle.LongRangePair(1000L, false, Long.MAX_VALUE - 1L, true),
+                new LongHyperRectangle.LongRangePair(1000L, false, Long.MAX_VALUE, true)));
+
+    FacetResult result = facets.getTopChildren(10, "field");
+    assertEquals(
+        """
+                        dim=field path=[] value=22 childCount=5
+                          less than (10, 11, 12) (10)
+                          less than or equal to (10, 11, 12) (11)
+                          over (90, 91, 92) (9)
+                          (90, 91, 92) or above (10)
+                          over (1000, 1000, 1000) (1)
+                        """,
+        result.toString());
+
+    // test getTopChildren(0, dim)
+    expectThrows(
+        IllegalArgumentException.class,
+        () -> {
+          facets.getTopChildren(0, "field");
+        });
+
+    r.close();
+    d.close();
+  }
+
+  public void testBasicDouble() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+
+    for (double l = 0; l < 100; l++) {
+      Document doc = new Document();
+      DoublePointFacetField field = new DoublePointFacetField("field", l, l + 1.0, l + 2.0);
+      doc.add(field);
+      w.addDocument(doc);
+    }
+
+    // Also add point with Long.MAX_VALUE
+    Document doc = new Document();
+    DoublePointFacetField field =
+        new DoublePointFacetField(
+            "field", Double.MAX_VALUE - 2.0, Double.MAX_VALUE - 1.0, Double.MAX_VALUE);
+    doc.add(field);
+    w.addDocument(doc);
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    FacetsCollector fc = s.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    Facets facets =
+        new HyperRectangleFacetCounts(
+            "field",
+            fc,
+            new DoubleHyperRectangle(
+                "less than (10, 11, 12)",
+                new DoubleHyperRectangle.DoubleRangePair(0.0, true, 10.0, false),
+                new DoubleHyperRectangle.DoubleRangePair(0.0, true, 11.0, false),
+                new DoubleHyperRectangle.DoubleRangePair(0.0, true, 12.0, false)),
+            new DoubleHyperRectangle(
+                "less than or equal to (10, 11, 12)",
+                new DoubleHyperRectangle.DoubleRangePair(0.0, true, 10.0, true),
+                new DoubleHyperRectangle.DoubleRangePair(0.0, true, 11.0, true),
+                new DoubleHyperRectangle.DoubleRangePair(0.0, true, 12.0, true)),
+            new DoubleHyperRectangle(
+                "over (90, 91, 92)",
+                new DoubleHyperRectangle.DoubleRangePair(90.0, false, 100.0, false),
+                new DoubleHyperRectangle.DoubleRangePair(91.0, false, 101.0, false),
+                new DoubleHyperRectangle.DoubleRangePair(92.0, false, 102.0, false)),
+            new DoubleHyperRectangle(
+                "(90, 91, 92) or above",
+                new DoubleHyperRectangle.DoubleRangePair(90.0, true, 100.0, false),
+                new DoubleHyperRectangle.DoubleRangePair(91.0, true, 101.0, false),
+                new DoubleHyperRectangle.DoubleRangePair(92.0, true, 102.0, false)),
+            new DoubleHyperRectangle(
+                "over (1000, 1000, 1000)",
+                new DoubleHyperRectangle.DoubleRangePair(
+                    1000.0, false, Double.MAX_VALUE - 2.0, true),
+                new DoubleHyperRectangle.DoubleRangePair(
+                    1000.0, false, Double.MAX_VALUE - 1.0, true),
+                new DoubleHyperRectangle.DoubleRangePair(1000.0, false, Double.MAX_VALUE, true)));
+
+    FacetResult result = facets.getTopChildren(10, "field");
+    assertEquals(
+        """
+                        dim=field path=[] value=22 childCount=5

Review Comment:
   Changed to test the actual `FacetResult` values rather than the `toString()`



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.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 by 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)

Review Comment:
   I think what you say makes sense, but in the rest of the facets API, `FacetsCollector` params are called hits. Here is an [example in SSDVFacetCounts](https://github.com/apache/lucene/blob/a071180a806d1bb7ae11ae30a07e43e452bea810/lucene/facet/src/java/org/apache/lucene/facet/sortedset/SortedSetDocValuesFacetCounts.java#L89). I think for now to keep consistency, I will leave named as is for now.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.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 by 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)

Review Comment:
   Yeah so I actually just added that parameter there in order to differentiate between the `public` and `private` constructors or they would have the same variable list/method signature. I needed to do this so I could init the `protected` fields as `final` while enforcing that all hyper rectangles passed in were of the same type without having to loop through them. I feel that this is a pretty bad solution though so maybe there's a better way to do this?



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.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 by 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    this.field = field;
+    this.hyperRectangles = hyperRectangles;
+    this.dims = hyperRectangles[0].dims;
+    for (HyperRectangle hyperRectangle : hyperRectangles) {
+      assert hyperRectangle.dims == this.dims
+          : "All hyper rectangles must be the same dimensionality";
+    }
+    this.counts = new int[hyperRectangles.length];
+    count(field, hits.getMatchingDocs());
+  }
+
+  /** 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; ) {
+        if (binaryDocValues.advanceExact(doc)) {
+          long[] point = LongPoint.unpack(binaryDocValues.binaryValue());
+          assert point.length == dims : "Point dimension is incompatible with hyper rectangle";

Review Comment:
   Added this to message.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangle.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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;
+
+/** Holds the name and the number of dims for a HyperRectangle */
+public abstract class HyperRectangle {
+  /** Label that identifies this range. */
+  public final String label;
+
+  /** How many dimensions this hyper rectangle has (IE: a regular rectangle would have dims=2) */
+  public final int dims;
+
+  /** Sole constructor. */
+  protected HyperRectangle(String label, int dims) {
+    if (label == null) {
+      throw new NullPointerException("label must not be null");
+    }
+    this.label = label;
+    this.dims = dims;

Review Comment:
   Added validity check.



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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r875458505


##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.util.Arrays;
+import org.apache.lucene.util.NumericUtils;
+
+/** Stores a hyper rectangle as an array of DoubleRangePairs */
+public class DoubleHyperRectangle extends HyperRectangle {
+
+  /** Creates DoubleHyperRectangle */
+  public DoubleHyperRectangle(String label, DoubleRangePair... pairs) {
+    super(label, convertToLongRangePairArray(pairs));
+  }
+
+  private static LongRangePair[] convertToLongRangePairArray(DoubleRangePair... pairs) {

Review Comment:
   nit: I find `Array` redundant, maybe `convertToLongRangePairs`? Or `toLongRangePairs`?



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert isHyperRectangleDimsConsistent(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 isHyperRectangleDimsConsistent(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; ) {
+        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);

Review Comment:
   nit: Now that it's part of `HyperRectangle` you can reference `LongRangePair` directly, but if you prefer to leave it for clarity I'm ok with that.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert isHyperRectangleDimsConsistent(hyperRectangles)

Review Comment:
   nit: `is` or `are`? I think we're referring to the plural rectangles and dims? So `areHyperRectangleDimsConsistent`?



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.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 by 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)

Review Comment:
   Yes I think a single constructor with `HyperRectangle...` is clear enough. Not sure we need to protect a user from passing a mix of Long/Double rectangles, and even if they do, we convert them to Long anyway ... I vote for keeping just this variant.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert isHyperRectangleDimsConsistent(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 isHyperRectangleDimsConsistent(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; ) {
+        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++;
+          }
+        }
+        doc = it.nextDoc();
+      }
+    }
+  }
+
+  @Override
+  public FacetResult getTopChildren(int topN, String dim, String... path) throws IOException {
+    validateTopN(topN);
+    if (field.equals(dim) == false) {
+      throw new IllegalArgumentException(
+          "invalid dim \"" + dim + "\"; should be \"" + field + "\"");
+    }
+    if (path.length != 0) {

Review Comment:
   nit: add `null` check? Do we do this elsewhere?



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangle.java:
##########
@@ -0,0 +1,100 @@
+/*
+ * 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;
+
+/** Holds the name and the number of dims for a HyperRectangle */
+public abstract class HyperRectangle {
+  /** Label that identifies this range. */
+  public final String label;
+
+  /** How many dimensions this hyper rectangle has (IE: a regular rectangle would have dims=2) */
+  public final int dims;
+
+  /** All subclasses should store pairs as comparable longs */
+  protected final LongRangePair[] pairs;
+
+  /** Sole constructor. */
+  protected HyperRectangle(String label, LongRangePair... pairs) {
+    if (label == null) {
+      throw new IllegalArgumentException("label must not be null");
+    }
+    if (pairs == null || pairs.length == 0) {
+      throw new IllegalArgumentException("Pairs cannot be null or empty");
+    }
+    this.label = label;
+    this.dims = pairs.length;
+    this.pairs = pairs;
+  }
+
+  /**
+   * Returns comparable long range for a provided dim
+   *
+   * @param dim dimension of the request range
+   * @return The comparable long version of the requested range
+   */
+  public LongRangePair getComparableDimRange(int dim) {
+    return pairs[dim];
+  }
+
+  /** Defines a single range in a HyperRectangle */
+  public static class LongRangePair {
+    /** Inclusive min */
+    public final long min;
+
+    /** Inclusive max */
+    public final long max;
+
+    /**
+     * Creates a LongRangePair, very similar to the constructor of {@link
+     * org.apache.lucene.facet.range.LongRange}
+     *
+     * @param minIn Min value of pair
+     * @param minInclusive If minIn is inclusive
+     * @param maxIn Max value of pair
+     * @param maxInclusive If maxIn is inclusive
+     */
+    public LongRangePair(long minIn, boolean minInclusive, long maxIn, boolean maxInclusive) {
+      if (!minInclusive) {
+        if (minIn != Long.MAX_VALUE) {
+          minIn++;
+        } else {
+          throw new IllegalArgumentException("Invalid min input, min=" + minIn);
+        }
+      }
+
+      if (!maxInclusive) {
+        if (maxIn != Long.MIN_VALUE) {
+          maxIn--;
+        } else {
+          throw new IllegalArgumentException("Invalid max input, max=" + maxIn);
+        }
+      }
+
+      if (minIn > maxIn) {
+        throw new IllegalArgumentException("Minimum cannot be greater than maximum");

Review Comment:
   I think here including the value of min/max in the message will be useful to debug.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert isHyperRectangleDimsConsistent(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 isHyperRectangleDimsConsistent(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; ) {
+        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++;
+          }
+        }
+        doc = it.nextDoc();
+      }
+    }
+  }
+
+  @Override
+  public FacetResult getTopChildren(int topN, String dim, String... path) throws IOException {
+    validateTopN(topN);
+    if (field.equals(dim) == false) {
+      throw new IllegalArgumentException(
+          "invalid dim \"" + dim + "\"; should be \"" + field + "\"");
+    }
+    if (path.length != 0) {
+      throw new IllegalArgumentException("path.length should be 0");

Review Comment:
   Typo: `path.length should not be 0`? Or `Must specify at least one path component`?



##########
lucene/core/src/java/org/apache/lucene/document/LongPoint.java:
##########
@@ -117,6 +117,25 @@ public static BytesRef pack(long... point) {
     return new BytesRef(packed);
   }
 
+  /**
+   * Unpack a BytesRef into a long point
+   *
+   * @param bytesRef BytesRef Value
+   * @throws IllegalArgumentException the value is null
+   */
+  public static long[] unpack(BytesRef bytesRef) {

Review Comment:
   Yeah OK I understand the reasoning. Technically we could get around that by counting the number of valid points per rectangle and only increase the count of a rectangle if all points were accepted, but that would introduce a temporary count array and I'm not sure it's better than unpacking to a `long[]`, so let's leave the code as-is.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert isHyperRectangleDimsConsistent(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 isHyperRectangleDimsConsistent(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; ) {
+        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++;
+          }
+        }
+        doc = it.nextDoc();
+      }
+    }
+  }
+
+  @Override
+  public FacetResult getTopChildren(int topN, String dim, String... path) throws IOException {
+    validateTopN(topN);
+    if (field.equals(dim) == false) {
+      throw new IllegalArgumentException(
+          "invalid dim \"" + dim + "\"; should be \"" + field + "\"");
+    }
+    if (path.length != 0) {
+      throw new IllegalArgumentException("path.length should be 0");
+    }
+    LabelAndValue[] labelValues = new LabelAndValue[counts.length];
+    for (int i = 0; i < counts.length; i++) {
+      labelValues[i] = new LabelAndValue(hyperRectangles[i].label, counts[i]);
+    }
+    return new FacetResult(dim, path, totCount, labelValues, labelValues.length);
+  }
+
+  @Override
+  public Number getSpecificValue(String dim, String... path) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public List<FacetResult> getAllDims(int topN) throws IOException {
+    validateTopN(topN);
+    return Collections.singletonList(getTopChildren(topN, field));

Review Comment:
   Here for example you just pass `field` as the path. Is it useful, or you did that because of the assertion that path length is not 0?



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert isHyperRectangleDimsConsistent(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 isHyperRectangleDimsConsistent(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; ) {
+        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++;
+          }
+        }
+        doc = it.nextDoc();

Review Comment:
   Q: can this be moved to the `for()` line itself? I think so?



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.util.Arrays;
+import org.apache.lucene.util.NumericUtils;
+
+/** Stores a hyper rectangle as an array of DoubleRangePairs */
+public class DoubleHyperRectangle extends HyperRectangle {
+
+  /** Creates DoubleHyperRectangle */
+  public DoubleHyperRectangle(String label, DoubleRangePair... pairs) {
+    super(label, convertToLongRangePairArray(pairs));
+  }
+
+  private static LongRangePair[] convertToLongRangePairArray(DoubleRangePair... pairs) {
+    if (pairs == null || pairs.length == 0) {
+      throw new IllegalArgumentException("Pairs cannot be null or empty");
+    }
+    return Arrays.stream(pairs).map(DoubleRangePair::toLongRangePair).toArray(LongRangePair[]::new);
+  }
+
+  /** Defines a single range in a DoubleHyperRectangle */
+  public static class DoubleRangePair {
+    /** Inclusive min */
+    public final double min;
+
+    /** Inclusive max */
+    public final double max;
+
+    /**
+     * Creates a DoubleRangePair, very similar to the constructor of {@link
+     * org.apache.lucene.facet.range.DoubleRange}
+     *
+     * @param minIn Min value of pair
+     * @param minInclusive If minIn is inclusive
+     * @param maxIn Max value of pair
+     * @param maxInclusive If maxIn is inclusive
+     */
+    public DoubleRangePair(double minIn, boolean minInclusive, double maxIn, boolean maxInclusive) {
+      if (Double.isNaN(minIn) || Double.isNaN(maxIn)) {
+        throw new IllegalArgumentException(
+            "min and max cannot be NaN: min=" + minIn + ", max=" + maxIn);
+      }
+
+      if (!minInclusive) {
+        minIn = Math.nextUp(minIn);
+      }
+
+      if (!maxInclusive) {
+        // Why no Math.nextDown?

Review Comment:
   Do we want to try answer this question? :) According to https://appdividend.com/2022/01/05/java-math-nextdown-function-example/:
   
   ```
   The nextDown() method is equivalent to nextAfter(d, Double.NEGATIVE_INFINITY) method.
   ```
   
   So I think you can just call `nextDown()`?



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert isHyperRectangleDimsConsistent(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 isHyperRectangleDimsConsistent(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; ) {
+        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++;
+          }
+        }
+        doc = it.nextDoc();
+      }
+    }
+  }
+
+  @Override
+  public FacetResult getTopChildren(int topN, String dim, String... path) throws IOException {
+    validateTopN(topN);
+    if (field.equals(dim) == false) {
+      throw new IllegalArgumentException(
+          "invalid dim \"" + dim + "\"; should be \"" + field + "\"");
+    }
+    if (path.length != 0) {
+      throw new IllegalArgumentException("path.length should be 0");

Review Comment:
   Now that I think of it, this method doesn't really care about the `path` right? It just passes it to `FacetResult`. Is there a reason to require a path at all? I understand it's part of the method contract, but since this method doesn't use it, I'm not sure we need to enforce at least one path component? WDYT?



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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1151141806

   @gsmiller, @mdmarshmallow I pushed another commit with the following changes:
   
   1. Assertion that all read facet sets have the same dimensionality (it's currently an `assert` to not affect runtime performance, but I think that if all is Kosher, most of these checks will be predicted correctly by the CPU?)
   2. Added another `matches(long[])` API so we can have a look at both variants. I also added a `countLongs` to `MFSC` and a TEMP constructor parameter to count either bytes or longs. After we decide whether to stick w/ the `long[]` or `byte[]` API we'll remove the unneeded variant.
   
   Tests pass (I verified both bytes and long counting logic were executed). I think that if we initialize the `long[]` once there's no performance overhead to working with longs? And the impls are much shorter and simpler?
   
   Anyway, now we have both versions to review, hopefully it will ease making a decision :).


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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1159391170

   > I don't think so? `FSD` is currently about decoding the encoded `byte[]` into a `long[]` for `FacetSetMatcher` purposes.
   
   Strike that, you're absolutely right! I think tests pass only because the values aren't actual `float/double`, I'll change that.


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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r897538192


##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSetsField.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.facetset;
+
+import java.util.Arrays;
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A {@link BinaryDocValuesField} which encodes a list of {@link FacetSet facet sets}. The encoding
+ * scheme consists of a packed {@code long[]} where the first value denotes the number of dimensions
+ * in all the sets, followed by each set's values.
+ *
+ * @lucene.experimental
+ */
+public class FacetSetsField extends BinaryDocValuesField {
+
+  /**
+   * Create a new FacetSets field.
+   *
+   * @param name field name
+   * @param facetSets the {@link FacetSet} to index in that field. All must have the same number of
+   *     dimensions
+   * @throws IllegalArgumentException if the field name is null or the given facet sets are invalid
+   */
+  public static FacetSetsField create(String name, FacetSet... facetSets) {
+    validateFacetSets(facetSets);
+
+    return new FacetSetsField(name, toPackedLongs(facetSets));
+  }
+
+  private FacetSetsField(String name, BytesRef value) {
+    super(name, value);
+  }
+
+  private static void validateFacetSets(FacetSet... facetSets) {

Review Comment:
   Good idea



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


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

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r903055633


##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSetsField.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.facetset;
+
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A {@link BinaryDocValuesField} which encodes a list of {@link FacetSet facet sets}. The encoding
+ * scheme consists of a packed {@code byte[]} where the first value denotes the number of dimensions
+ * in all the sets, followed by each set's values.
+ *
+ * @lucene.experimental
+ */
+public class FacetSetsField extends BinaryDocValuesField {
+
+  /**
+   * Create a new FacetSets field.
+   *
+   * @param name field name
+   * @param facetSets the {@link FacetSet facet sets} to index in that field. All must have the same
+   *     number of dimensions
+   * @throws IllegalArgumentException if the field name is null or the given facet sets are invalid
+   */
+  public static FacetSetsField create(String name, FacetSet... facetSets) {
+    if (facetSets == null || facetSets.length == 0) {
+      throw new IllegalArgumentException("FacetSets cannot be null or empty!");
+    }
+
+    return new FacetSetsField(name, toPackedValues(facetSets));
+  }
+
+  private FacetSetsField(String name, BytesRef value) {
+    super(name, value);
+  }
+
+  private static BytesRef toPackedValues(FacetSet... facetSets) {
+    int numDims = facetSets[0].dims;
+    Class<?> expectedClass = facetSets[0].getClass();
+    byte[] buf = new byte[Integer.BYTES + facetSets[0].sizePackedBytes() * facetSets.length];
+    IntPoint.encodeDimension(numDims, buf, 0);
+    int offset = Integer.BYTES;
+    for (FacetSet facetSet : facetSets) {
+      if (facetSet.dims != numDims) {
+        throw new IllegalArgumentException(
+            "All FacetSets must have the same number of dimensions. Expected "
+                + numDims
+                + " found "
+                + facetSet.dims);
+      }
+      // It doesn't make sense to index facet sets of different types in the same field
+      if (facetSet.getClass() != expectedClass) {

Review Comment:
   Took a look at this again and yeah, it doesn't make sense to generify here.



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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1163142014

   From my point of view, I think we're ready to ship this thing! Thanks @mdmarshmallow and @shaie!


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1165780069

   +1 to backporting to 9.x. I think we're ready to merge as far as I'm concerned. @shaie I'll leave it to you to merge and backport, assuming you also feel we're good-to-go here? If you'd prefer I merge/backport, I'm happy to help out as well. Thanks!


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r903051691


##########
lucene/demo/src/java/org/apache/lucene/demo/facet/CustomFacetSetExample.java:
##########
@@ -0,0 +1,303 @@
+/*
+ * 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.time.LocalDate;
+import java.time.ZoneOffset;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.document.*;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollectorManager;
+import org.apache.lucene.facet.facetset.*;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.store.ByteBuffersDirectory;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * Shows usage of indexing and searching {@link FacetSetsField} with a custom {@link FacetSet}
+ * implementation. Unlike the out of the box {@link FacetSet} implementations, this example shows
+ * how to mix and match dimensions of different types, as well as implementing a custom {@link
+ * FacetSetMatcher}.
+ */
+public class CustomFacetSetExample {
+
+  private static final long MAY_SECOND_2022 = date("2022-05-02");
+  private static final long JUNE_SECOND_2022 = date("2022-06-02");
+  private static final long JULY_SECOND_2022 = date("2022-07-02");
+  private static final float HUNDRED_TWENTY_DEGREES = fahrenheitToCelsius(120);
+  private static final float HUNDRED_DEGREES = fahrenheitToCelsius(100);
+  private static final float EIGHTY_DEGREES = fahrenheitToCelsius(80);
+
+  private final Directory indexDir = new ByteBuffersDirectory();
+
+  /** Empty constructor */
+  public CustomFacetSetExample() {}
+
+  /** Build the example index. */
+  private void index() throws IOException {
+    IndexWriter indexWriter =
+        new IndexWriter(
+            indexDir, new IndexWriterConfig(new WhitespaceAnalyzer()).setOpenMode(OpenMode.CREATE));
+
+    // Every document holds the temperature measures for a City by Date
+
+    Document doc = new Document();
+    doc.add(new StringField("city", "city1", Field.Store.YES));
+    doc.add(
+        FacetSetsField.create(
+            "temperature",
+            new TemperatureReadingFacetSet(MAY_SECOND_2022, HUNDRED_DEGREES),
+            new TemperatureReadingFacetSet(JUNE_SECOND_2022, EIGHTY_DEGREES),
+            new TemperatureReadingFacetSet(JULY_SECOND_2022, HUNDRED_TWENTY_DEGREES)));
+    indexWriter.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new StringField("city", "city2", Field.Store.YES));
+    doc.add(
+        FacetSetsField.create(
+            "temperature",
+            new TemperatureReadingFacetSet(MAY_SECOND_2022, EIGHTY_DEGREES),
+            new TemperatureReadingFacetSet(JUNE_SECOND_2022, HUNDRED_DEGREES),
+            new TemperatureReadingFacetSet(JULY_SECOND_2022, HUNDRED_TWENTY_DEGREES)));
+    indexWriter.addDocument(doc);
+
+    indexWriter.close();
+  }
+
+  /** Counting documents which exactly match a given {@link FacetSet}. */
+  private List<FacetResult> exactMatching() throws IOException {
+    DirectoryReader indexReader = DirectoryReader.open(indexDir);
+    IndexSearcher searcher = new IndexSearcher(indexReader);
+
+    // MatchAllDocsQuery is for "browsing" (counts facets
+    // for all non-deleted docs in the index); normally
+    // you'd use a "normal" query:
+    FacetsCollector fc = searcher.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    // Count both "Publish Date" and "Author" dimensions
+    Facets facets =
+        new MatchingFacetSetsCounts(
+            "temperature",
+            fc,
+            TemperatureReadingFacetSet::decodeTemperatureReading,
+            new ExactFacetSetMatcher(
+                "May 2022 (100f)",
+                new TemperatureReadingFacetSet(MAY_SECOND_2022, HUNDRED_DEGREES)),
+            new ExactFacetSetMatcher(
+                "July 2022 (120f)",
+                new TemperatureReadingFacetSet(JULY_SECOND_2022, HUNDRED_TWENTY_DEGREES)));
+
+    // Retrieve results
+    List<FacetResult> results = Collections.singletonList(facets.getTopChildren(10, "temperature"));

Review Comment:
   Is there a reason all these methods return `List<FacetResult>` and not just `FacetResult`? It seems like we're wrapping with `Collections#singletonList` in all cases and then just "unwrapping" the single result in all cases.



##########
lucene/demo/src/java/org/apache/lucene/demo/facet/CustomFacetSetExample.java:
##########
@@ -0,0 +1,303 @@
+/*
+ * 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.time.LocalDate;
+import java.time.ZoneOffset;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.document.*;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollectorManager;
+import org.apache.lucene.facet.facetset.*;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.store.ByteBuffersDirectory;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * Shows usage of indexing and searching {@link FacetSetsField} with a custom {@link FacetSet}
+ * implementation. Unlike the out of the box {@link FacetSet} implementations, this example shows
+ * how to mix and match dimensions of different types, as well as implementing a custom {@link
+ * FacetSetMatcher}.
+ */
+public class CustomFacetSetExample {
+
+  private static final long MAY_SECOND_2022 = date("2022-05-02");
+  private static final long JUNE_SECOND_2022 = date("2022-06-02");
+  private static final long JULY_SECOND_2022 = date("2022-07-02");
+  private static final float HUNDRED_TWENTY_DEGREES = fahrenheitToCelsius(120);
+  private static final float HUNDRED_DEGREES = fahrenheitToCelsius(100);
+  private static final float EIGHTY_DEGREES = fahrenheitToCelsius(80);
+
+  private final Directory indexDir = new ByteBuffersDirectory();
+
+  /** Empty constructor */
+  public CustomFacetSetExample() {}
+
+  /** Build the example index. */
+  private void index() throws IOException {
+    IndexWriter indexWriter =
+        new IndexWriter(
+            indexDir, new IndexWriterConfig(new WhitespaceAnalyzer()).setOpenMode(OpenMode.CREATE));
+
+    // Every document holds the temperature measures for a City by Date
+
+    Document doc = new Document();
+    doc.add(new StringField("city", "city1", Field.Store.YES));
+    doc.add(
+        FacetSetsField.create(
+            "temperature",
+            new TemperatureReadingFacetSet(MAY_SECOND_2022, HUNDRED_DEGREES),
+            new TemperatureReadingFacetSet(JUNE_SECOND_2022, EIGHTY_DEGREES),
+            new TemperatureReadingFacetSet(JULY_SECOND_2022, HUNDRED_TWENTY_DEGREES)));
+    indexWriter.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new StringField("city", "city2", Field.Store.YES));
+    doc.add(
+        FacetSetsField.create(
+            "temperature",
+            new TemperatureReadingFacetSet(MAY_SECOND_2022, EIGHTY_DEGREES),
+            new TemperatureReadingFacetSet(JUNE_SECOND_2022, HUNDRED_DEGREES),
+            new TemperatureReadingFacetSet(JULY_SECOND_2022, HUNDRED_TWENTY_DEGREES)));
+    indexWriter.addDocument(doc);
+
+    indexWriter.close();
+  }
+
+  /** Counting documents which exactly match a given {@link FacetSet}. */
+  private List<FacetResult> exactMatching() throws IOException {
+    DirectoryReader indexReader = DirectoryReader.open(indexDir);
+    IndexSearcher searcher = new IndexSearcher(indexReader);
+
+    // MatchAllDocsQuery is for "browsing" (counts facets
+    // for all non-deleted docs in the index); normally
+    // you'd use a "normal" query:
+    FacetsCollector fc = searcher.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    // Count both "Publish Date" and "Author" dimensions

Review Comment:
   oops



##########
lucene/demo/src/java/org/apache/lucene/demo/facet/CustomFacetSetExample.java:
##########
@@ -0,0 +1,303 @@
+/*
+ * 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.time.LocalDate;
+import java.time.ZoneOffset;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.document.*;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollectorManager;
+import org.apache.lucene.facet.facetset.*;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.store.ByteBuffersDirectory;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * Shows usage of indexing and searching {@link FacetSetsField} with a custom {@link FacetSet}
+ * implementation. Unlike the out of the box {@link FacetSet} implementations, this example shows
+ * how to mix and match dimensions of different types, as well as implementing a custom {@link
+ * FacetSetMatcher}.
+ */
+public class CustomFacetSetExample {
+
+  private static final long MAY_SECOND_2022 = date("2022-05-02");
+  private static final long JUNE_SECOND_2022 = date("2022-06-02");
+  private static final long JULY_SECOND_2022 = date("2022-07-02");
+  private static final float HUNDRED_TWENTY_DEGREES = fahrenheitToCelsius(120);
+  private static final float HUNDRED_DEGREES = fahrenheitToCelsius(100);
+  private static final float EIGHTY_DEGREES = fahrenheitToCelsius(80);
+
+  private final Directory indexDir = new ByteBuffersDirectory();
+
+  /** Empty constructor */
+  public CustomFacetSetExample() {}
+
+  /** Build the example index. */
+  private void index() throws IOException {
+    IndexWriter indexWriter =
+        new IndexWriter(
+            indexDir, new IndexWriterConfig(new WhitespaceAnalyzer()).setOpenMode(OpenMode.CREATE));
+
+    // Every document holds the temperature measures for a City by Date
+
+    Document doc = new Document();
+    doc.add(new StringField("city", "city1", Field.Store.YES));
+    doc.add(
+        FacetSetsField.create(
+            "temperature",
+            new TemperatureReadingFacetSet(MAY_SECOND_2022, HUNDRED_DEGREES),
+            new TemperatureReadingFacetSet(JUNE_SECOND_2022, EIGHTY_DEGREES),
+            new TemperatureReadingFacetSet(JULY_SECOND_2022, HUNDRED_TWENTY_DEGREES)));
+    indexWriter.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new StringField("city", "city2", Field.Store.YES));
+    doc.add(
+        FacetSetsField.create(
+            "temperature",
+            new TemperatureReadingFacetSet(MAY_SECOND_2022, EIGHTY_DEGREES),
+            new TemperatureReadingFacetSet(JUNE_SECOND_2022, HUNDRED_DEGREES),
+            new TemperatureReadingFacetSet(JULY_SECOND_2022, HUNDRED_TWENTY_DEGREES)));
+    indexWriter.addDocument(doc);
+
+    indexWriter.close();
+  }
+
+  /** Counting documents which exactly match a given {@link FacetSet}. */
+  private List<FacetResult> exactMatching() throws IOException {
+    DirectoryReader indexReader = DirectoryReader.open(indexDir);
+    IndexSearcher searcher = new IndexSearcher(indexReader);
+
+    // MatchAllDocsQuery is for "browsing" (counts facets
+    // for all non-deleted docs in the index); normally
+    // you'd use a "normal" query:
+    FacetsCollector fc = searcher.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    // Count both "Publish Date" and "Author" dimensions
+    Facets facets =
+        new MatchingFacetSetsCounts(
+            "temperature",
+            fc,
+            TemperatureReadingFacetSet::decodeTemperatureReading,
+            new ExactFacetSetMatcher(
+                "May 2022 (100f)",
+                new TemperatureReadingFacetSet(MAY_SECOND_2022, HUNDRED_DEGREES)),
+            new ExactFacetSetMatcher(
+                "July 2022 (120f)",
+                new TemperatureReadingFacetSet(JULY_SECOND_2022, HUNDRED_TWENTY_DEGREES)));
+
+    // Retrieve results
+    List<FacetResult> results = Collections.singletonList(facets.getTopChildren(10, "temperature"));
+
+    indexReader.close();
+
+    return results;
+  }
+
+  /** Counting documents which match a certain degrees value for any date. */
+  private List<FacetResult> rangeMatching() throws IOException {
+    DirectoryReader indexReader = DirectoryReader.open(indexDir);
+    IndexSearcher searcher = new IndexSearcher(indexReader);
+
+    // MatchAllDocsQuery is for "browsing" (counts facets
+    // for all non-deleted docs in the index); normally
+    // you'd use a "normal" query:
+    FacetsCollector fc = searcher.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    // Count both "Publish Date" and "Author" dimensions
+    Facets facets =
+        new MatchingFacetSetsCounts(
+            "temperature",
+            fc,
+            TemperatureReadingFacetSet::decodeTemperatureReading,
+            new RangeFacetSetMatcher(
+                "Eighty to Hundred Degrees",
+                RangeFacetSetMatcher.DimRange.fromLongs(Long.MIN_VALUE, true, Long.MAX_VALUE, true),
+                RangeFacetSetMatcher.DimRange.fromFloats(
+                    EIGHTY_DEGREES, true, HUNDRED_DEGREES, true)));
+
+    // Retrieve results
+    List<FacetResult> results = Collections.singletonList(facets.getTopChildren(10, "temperature"));
+
+    indexReader.close();
+
+    return results;
+  }
+
+  /**
+   * Like {@link #rangeMatching()}, however this example demonstrates a custom {@link
+   * FacetSetMatcher} which only considers certain dimensions (in this case only the temperature
+   * one).
+   */
+  private List<FacetResult> customRangeMatching() throws IOException {
+    DirectoryReader indexReader = DirectoryReader.open(indexDir);
+    IndexSearcher searcher = new IndexSearcher(indexReader);
+
+    // MatchAllDocsQuery is for "browsing" (counts facets
+    // for all non-deleted docs in the index); normally
+    // you'd use a "normal" query:
+    FacetsCollector fc = searcher.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    // Count both "Publish Date" and "Author" dimensions

Review Comment:
   oops :)



##########
lucene/demo/src/java/org/apache/lucene/demo/facet/CustomFacetSetExample.java:
##########
@@ -0,0 +1,303 @@
+/*
+ * 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.time.LocalDate;
+import java.time.ZoneOffset;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.document.*;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollectorManager;
+import org.apache.lucene.facet.facetset.*;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.store.ByteBuffersDirectory;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * Shows usage of indexing and searching {@link FacetSetsField} with a custom {@link FacetSet}
+ * implementation. Unlike the out of the box {@link FacetSet} implementations, this example shows
+ * how to mix and match dimensions of different types, as well as implementing a custom {@link
+ * FacetSetMatcher}.
+ */
+public class CustomFacetSetExample {
+
+  private static final long MAY_SECOND_2022 = date("2022-05-02");
+  private static final long JUNE_SECOND_2022 = date("2022-06-02");
+  private static final long JULY_SECOND_2022 = date("2022-07-02");
+  private static final float HUNDRED_TWENTY_DEGREES = fahrenheitToCelsius(120);
+  private static final float HUNDRED_DEGREES = fahrenheitToCelsius(100);
+  private static final float EIGHTY_DEGREES = fahrenheitToCelsius(80);
+
+  private final Directory indexDir = new ByteBuffersDirectory();
+
+  /** Empty constructor */
+  public CustomFacetSetExample() {}
+
+  /** Build the example index. */
+  private void index() throws IOException {
+    IndexWriter indexWriter =
+        new IndexWriter(
+            indexDir, new IndexWriterConfig(new WhitespaceAnalyzer()).setOpenMode(OpenMode.CREATE));
+
+    // Every document holds the temperature measures for a City by Date
+
+    Document doc = new Document();
+    doc.add(new StringField("city", "city1", Field.Store.YES));
+    doc.add(
+        FacetSetsField.create(
+            "temperature",
+            new TemperatureReadingFacetSet(MAY_SECOND_2022, HUNDRED_DEGREES),
+            new TemperatureReadingFacetSet(JUNE_SECOND_2022, EIGHTY_DEGREES),
+            new TemperatureReadingFacetSet(JULY_SECOND_2022, HUNDRED_TWENTY_DEGREES)));
+    indexWriter.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new StringField("city", "city2", Field.Store.YES));
+    doc.add(
+        FacetSetsField.create(
+            "temperature",
+            new TemperatureReadingFacetSet(MAY_SECOND_2022, EIGHTY_DEGREES),
+            new TemperatureReadingFacetSet(JUNE_SECOND_2022, HUNDRED_DEGREES),
+            new TemperatureReadingFacetSet(JULY_SECOND_2022, HUNDRED_TWENTY_DEGREES)));
+    indexWriter.addDocument(doc);
+
+    indexWriter.close();
+  }
+
+  /** Counting documents which exactly match a given {@link FacetSet}. */
+  private List<FacetResult> exactMatching() throws IOException {
+    DirectoryReader indexReader = DirectoryReader.open(indexDir);
+    IndexSearcher searcher = new IndexSearcher(indexReader);
+
+    // MatchAllDocsQuery is for "browsing" (counts facets
+    // for all non-deleted docs in the index); normally
+    // you'd use a "normal" query:
+    FacetsCollector fc = searcher.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    // Count both "Publish Date" and "Author" dimensions
+    Facets facets =
+        new MatchingFacetSetsCounts(
+            "temperature",
+            fc,
+            TemperatureReadingFacetSet::decodeTemperatureReading,
+            new ExactFacetSetMatcher(
+                "May 2022 (100f)",
+                new TemperatureReadingFacetSet(MAY_SECOND_2022, HUNDRED_DEGREES)),
+            new ExactFacetSetMatcher(
+                "July 2022 (120f)",
+                new TemperatureReadingFacetSet(JULY_SECOND_2022, HUNDRED_TWENTY_DEGREES)));
+
+    // Retrieve results
+    List<FacetResult> results = Collections.singletonList(facets.getTopChildren(10, "temperature"));
+
+    indexReader.close();
+
+    return results;
+  }
+
+  /** Counting documents which match a certain degrees value for any date. */
+  private List<FacetResult> rangeMatching() throws IOException {
+    DirectoryReader indexReader = DirectoryReader.open(indexDir);
+    IndexSearcher searcher = new IndexSearcher(indexReader);
+
+    // MatchAllDocsQuery is for "browsing" (counts facets
+    // for all non-deleted docs in the index); normally
+    // you'd use a "normal" query:
+    FacetsCollector fc = searcher.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    // Count both "Publish Date" and "Author" dimensions

Review Comment:
   oops :)



##########
lucene/facet/docs/FacetSets.adoc:
##########
@@ -0,0 +1,130 @@
+= FacetSets Overview
+:toc:
+
+This document describes the `FacetSets` capability, which allows to aggregate on multidimensional values. It starts
+with outlining a few example use cases to showcase the motivation for this capability and follows with an API
+walk through.
+
+== Motivation
+
+[#movie-actors]
+=== Movie Actors DB
+
+Suppose that you want to build a search engine for movie actors which allows you to search for actors by name and see
+movie titles they appeared in. You might want to index standard fields such as `actorName`, `genre` and `releaseYear`
+which will let you search by the actor's name or see all actors who appeared in movies during 2021. Similarly, you can
+index facet fields that will let you aggregate by “Genre” and “Year” so that you can show how many actors appeared in
+each year or genre. Few example documents:
+
+[source]
+----
+{ "name": "Tom Hanks", "genre": ["Comedy", "Drama", …], "year": [1988, 2000,…] }
+{ "name": "Harrison Ford", "genre": ["Action", "Adventure", …], "year": [1977, 1981, …] }
+----
+
+However, these facet fields do not allow you to show the following aggregation:
+
+.Number of Actors performing in movies by Genre and Year
+[cols="4*"]
+|===
+|           | 2020 | 2021 | 2022
+| Thriller  | 121  | 43   | 97
+| Action    | 145  | 52   | 130
+| Adventure | 87   | 21   | 32
+|===
+
+The reason is that each “genre” or “releaseYear” facet field is indexed in its own data structure, and therefore if an
+actor appeared in a "Thriller" movie in "2020" and "Action" movie in "2021", there's no way for you to tell that they
+didn't appear in an "Action" movie in "2020".
+
+[#automotive-parts]
+=== Automotive Parts Store
+
+Say you're building a search engine for an automotive parts store where customers can search for different car parts.
+For simplicity let's assume that each item in the catalog contains a searchable “type” field and “car model” it fits
+which consists of two separate fields: “manufacturer” and “year”. This lets you search for parts by their type as well
+as filter parts that fit only a certain manufacturer or year. Few example documents:
+
+[source]
+----
+{
+  "type": "Wiper Blades V1",
+  "models": [
+    { "manufaturer": "Ford", "year": 2010 },
+    { "manufacturer": "Chevy", "year": 2011 }
+  ]
+}
+{
+  "type": "Wiper Blades V2",
+  "models": [
+    { "manufaturer": "Ford", "year": 2011 },
+    { "manufacturer": "Chevy", "year": 2010 }
+  ]
+}
+----
+
+By breaking up the "models" field into its sub-fields "manufacturer" and "year", you can easily aggregate on parts that
+fit a certain manufacturer or year. However, if a user would like to aggregate on parts that can fit either a "Ford
+2010" or "Chevy 2011", then aggregating on the sub-fields will lead to a wrong count of 2 (in the above example) instead
+of 1.
+
+[#movie-awards]
+=== Movie Awards
+
+To showcase a 3-D multidimensional aggregation, lets expand the <<movie-actors>> example with awards an actor has
+received over the years. For this aggregation we will use four dimensions: Award Type ("Oscar", "Grammy", "Emmy"),
+Award Category ("Best Actor", "Best Supporting Actress"), Year and Genre. One interesting aggregation is to show how
+many "Best Actor" vs "Best Supporting Actor" awards one has received in the "Oscar" or "Emmy" for each year. Another
+aggregation is slicing the number of these awards by Genre over all the years.
+
+Building on these examples, one might be able to come up with an interesting use case for an N-dimensional aggregation
+(where `N > 3`). The higher `N` is, the harder it is to aggregate all the dimensions correctly and efficiently without
+`FacetSets`.
+
+== FacetSets API
+
+The `facetset` package consists of few components which allow you to index and aggregate multidimensional facet sets:
+
+=== FacetSet
+
+Holds a set of facet dimension values. Implementations are required to convert the dimensions into comparable long
+representation, as well can implement how the values are packed (encoded). The package offers four implementations:
+`Int/Float/Long/DoubleFacetSet` for `int`, `float`, `long` and `double` values respectively. You can also look at
+`org.apache.lucene.demo.facet.CustomFacetSetExample` in the `lucene/demo` package for a custom implementation of a
+`FacetSet`.
+
+=== FacetSetsField
+
+A `BinaryDocValues` field which lets you index a list of `FacetSet`. This field can be added to a document only once, so
+you will need to construct all the facet sets in advance.
+
+=== FacetSetMatcher
+
+Responsible for matching an encoded `FacetSet` against a given criteria. For example, `ExactFacetSetMatcher` only
+considers an encoded facet set as a match if all dimension values are equal to a given one. `RangeFacetSetMatcher`
+considers an encoded facet set as a match if all dimension values fall within predefined ranges. You can also look at
+`org.apache.lucene.demo.facet.CustomFacetSetExample` in the `lucene/demo` package for a custom implementation of a
+`FacetSetMatcher`.
+
+=== FacetSetDecoder

Review Comment:
   Maybe a note here that the "off-the-shelf" int/long/float/double FacetSet types can use `fromLong` / `fromInt` methods that are provided? 



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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1159374588

   > My only suggestion here might be to rename `LongRange` to just `Range`
   
   Yeah `LongRange` now feels like there are missing `Int/Float/DoubleRange` which is not the case. But maybe in order to give it a more purposeful name we can name it `Dim/DimensionRange`?
   
   
   
   > Also, should we add `double` and `float` methods to `FacetSetDecoder`?
   
   I don't think so? `FSD` is currently about decoding the encoded `byte[]` into a `long[]` for `FacetSetMatcher` purposes.  I assume you're thinking about a user-level API which can decode the values back into a `FacetSet` right? Feels to me like we can do it later too, and I think we'll need a diff API for that, maybe a `FacetSetReader` with `FacetSet[] fromBytes(BytesRef)` or maybe add to `FacetSet` an `unpackValues` method? I prefer though that we focus in this PR on the indexing + matching, to get this PR to completion and also since it doesn't currently feel to me like a necessary API to use facet sets. 


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


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

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r901174337


##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSetsField.java:
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.facetset;
+
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A {@link BinaryDocValuesField} which encodes a list of {@link FacetSet facet sets}. The encoding
+ * scheme consists of a packed {@code byte[]} where the first value denotes the number of dimensions
+ * in all the sets, followed by each set's values.
+ *
+ * @lucene.experimental
+ */
+public class FacetSetsField extends BinaryDocValuesField {
+
+  /**
+   * Create a new FacetSets field.
+   *
+   * @param name field name
+   * @param facetSets the {@link FacetSet facet sets} to index in that field. All must have the same
+   *     number of dimensions
+   * @throws IllegalArgumentException if the field name is null or the given facet sets are invalid
+   */
+  public static FacetSetsField create(String name, FacetSet... facetSets) {
+    if (facetSets == null || facetSets.length == 0) {
+      throw new IllegalArgumentException("FacetSets cannot be null or empty!");
+    }
+
+    return new FacetSetsField(name, toPackedValues(facetSets));
+  }
+
+  private FacetSetsField(String name, BytesRef value) {
+    super(name, value);
+  }
+
+  private static BytesRef toPackedValues(FacetSet... facetSets) {
+    int numDims = facetSets[0].dims;
+    Class<?> expectedClass = facetSets[0].getClass();
+    byte[] buf = new byte[Integer.BYTES + facetSets[0].sizePackedBytes() * facetSets.length];
+    IntPoint.encodeDimension(numDims, buf, 0);
+    int offset = Integer.BYTES;
+    for (FacetSet facetSet : facetSets) {
+      if (facetSet.dims != numDims) {
+        throw new IllegalArgumentException(
+            "All FacetSets must have the same number of dimensions. Expected "
+                + numDims
+                + " found "
+                + facetSet.dims);
+      }
+      // It doesn't make sense to index facet sets of different types in the same field
+      if (facetSet.getClass() != expectedClass) {

Review Comment:
   Thoughts on using generics here to enforce this at compile time?



##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/package-info.java:
##########
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+
+/** Provides FacetSets faceting capabilities. */

Review Comment:
   Maybe make this slightly more descriptive? "Provides FacetSets faceting capabilities which allows users to facet on on high dimensional field values. See FacetSets.adoc in the docs package for more information on usage." Or something like that.



##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/RangeFacetSetMatcher.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.facetset;
+
+import java.util.Arrays;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * A {@link FacetSetMatcher} which considers a set as a match if all dimensions fall within the
+ * given corresponding range.
+ *
+ * @lucene.experimental
+ */
+public class RangeFacetSetMatcher extends FacetSetMatcher {
+
+  private final long[] lowerRanges;
+  private final long[] upperRanges;
+
+  /**
+   * Constructs an instance to match facet sets with dimensions that fall within the given ranges.
+   */
+  public RangeFacetSetMatcher(String label, DimRange... dimRanges) {
+    super(label, getDims(dimRanges));
+    this.lowerRanges = Arrays.stream(dimRanges).mapToLong(range -> range.min).toArray();
+    this.upperRanges = Arrays.stream(dimRanges).mapToLong(range -> range.max).toArray();
+  }
+
+  @Override
+  public boolean matches(long[] dimValues) {
+    assert dimValues.length == dims
+        : "Encoded dimensions (dims="
+            + dimValues.length
+            + ") is incompatible with range dimensions (dims="
+            + dims
+            + ")";
+
+    for (int i = 0; i < dimValues.length; i++) {
+      if (dimValues[i] < lowerRanges[i]) {
+        // Doc's value is too low in this dimension
+        return false;
+      }
+      if (dimValues[i] > upperRanges[i]) {
+        // Doc's value is too high in this dimension
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private static int getDims(DimRange... dimRanges) {
+    if (dimRanges == null || dimRanges.length == 0) {
+      throw new IllegalArgumentException("dimRanges cannot be null or empty");
+    }
+    return dimRanges.length;
+  }
+
+  /**
+   * Creates a {@link DimRange} for the given min and max long values. This method is also suitable
+   * for int values.
+   */
+  public static DimRange fromLongs(long min, boolean minInclusive, long max, boolean maxInclusive) {

Review Comment:
   nit: Should these `from` functions go into the `DimRange` class itself? I think that it makes more sense that way to me personally.



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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r897535976


##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/ExactFacetSetMatcher.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.facetset;
+
+/**
+ * A {@link FacetSetMatcher} which considers a set as a match only if all dimension values are equal
+ * to the given one.
+ *
+ * @lucene.experimental
+ */
+public class ExactFacetSetMatcher extends FacetSetMatcher {
+
+  private final long[] values;
+
+  /** Constructs an instance to match the given facet set. */
+  public ExactFacetSetMatcher(String label, FacetSet facetSet) {
+    super(label, facetSet.values.length);
+    this.values = facetSet.values;
+  }
+
+  @Override
+  public boolean matches(long[] dimValues) {
+    assert dimValues.length == dims
+        : "Encoded dimensions (dims="
+            + dimValues.length
+            + ") is incompatible with FacetSet dimensions (dims="
+            + dims
+            + ")";
+
+    for (int i = 0; i < dimValues.length; i++) {
+      if (dimValues[i] != values[i]) {
+        // Field's dimension value is not equal to given dimension, the entire set is rejected
+        return false;
+      }
+    }
+    return true;

Review Comment:
   I thought we want to avoid calling other methods from such hot code, but yeah, `Arrays.equals` may even be more optimal. 👍 



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


[GitHub] [lucene] mdmarshmallow commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1152603748

   >Is this about providing enough information to optimize with KD/R-Trees? If that's the case, I don't think we want a matches(long[]) method right? We just need a way for the FSM instances to expose their bounding-boxes. I don't think we even need to store the original long[] array to do this.
   
   I just wanted to point out here that the reason I wanted a `long[]` API was to support KD/R-Trees, but if we don't need it to add support, then I think there is no reason for `long[]`. I feel like having 2 `matches` functions in this case would make the API unnecessarily complex. If the user is extending the FSM, then I feel like they could figure out the `byte[]` API?


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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1162752539

   I don't know if what I've done is OK, but the last commit that I pushed failed the distribution tests because the API of `Facets` has changed and introduced a new `abstract` method. So I rebased this branch on `upstream/main` and pushed, but it seems to have pushed a slew of commits?
   
   @mdmarshmallow I tried to do the rebase first on your `mdmarshmallow/main` but I don't have permissions, so perhaps that's something you could do?


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1161727452

   Thanks @shaie! I was away from my computer since Thursday but should have time to catch up on this today, respond to your comments and do another review pass. Agreed that we're close on this. Finish line is in sight! :)


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r879871471


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

Review Comment:
   That makes sense. I think leaving it `private` until there's a need is good.



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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1141887260

   @shaie thanks for providing the use case doc. Very helpful!
   
   As far as an API proposal, I really like the "facet set" concept for the actual `Facets` implementation. Longer-term, I'd be more in favor of keeping the new field more generic (e.g., a generic "points" doc values field that can be used for faceting but also "slow" queries). But making it more general also makes it harder to understand for users that are just trying to use this faceting functionality. If we end up proposing this as a "sandbox" module feature for now, then I'd +1 this idea of a "facet set" API. If we propose adding this to the core functionality though, I'd like to further discuss the pros/cons of how specific we make this new doc values field.
   
   Thanks again!


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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1142206707

   > Longer-term, I'd be more in favor of keeping the new field more generic (e.g., a generic "points" doc values field that can be used for faceting but also "slow" queries)
   
   There are two sides to FacetSets: indexing and aggregation. Well actually three: drill-down too. For indexing I think a generic `long... values` field make sense for FacetSets and perhaps other use cases unrelated to faceting. As long as one can use its encoding as `LongPoint.pack()` or any other future scheme that will be _generic_, then I don't see why not. However if that's all what this field will currently do, then I prefer that we start without it in the `facet` package. We can always extend this field in the future, rather than BDV, right? (as long as it keeps the same order of the values)
   
   For aggregation though, as I thought about the API more, I realized that a generic "aggregator" makes no sense. On the file-system it's just a `long[]` but how you aggregate this series is totally use-case dependent:
   
   * For the "Automotive Parts Store" example, you'd want a "Matching Facet Counts" which are given a list of `FacetSet` objects and count documents which have an exact matching set.
   * For the HyperRectangle faceting (what's currently proposed in this PR) you'd want a different Facet Counts object which ensures that a given point falls within a pair of `long` values. So here the underlying `long[]` is actually pair of longs, really both belong to the same "dimension" maybe, and however a "FacetSet" is matched is a different function than for the one above.
   * For N-dimensional FacetSets, a'la the "Movie Awards" example, one might index 5 dimensions for a Movie award, but choose to aggregate only on 3 of them. In that case you'll want a Facet matcher which knows in advance which dimensions it picks from the set to determine whether there's a match.
   * And lastly for the "Movie Actor DB" example, if you'd want to actually compute a matrix, then our `Facets` API is completely not suitable, since it's a flat single-dimension result type. I don't propose to extend it, only saying that if we wanted to compute N-dimensional results, we'd need another API.
     * Actually, this feature by itself is very cool, since it's an interesting challenge to compute a matrix of "interesting" values. You can't just pick the top "cells" since that will produce fragmented rows, so you have to actually pick "interesting rows/columns". But that's for a different PR.
   
   Therefore, while it looks like a generic `long[]` DV (unsorted!) field might be enough for indexing facet sets, for aggregations I think we'll have few APIs.
   
   > If we end up proposing this as a "sandbox" module feature for now, then I'd +1 this idea of a "facet set" API.
   
   I am personally for developing it directly in the facet module. If we add `@lucene.experimental` to the classes, along with a "NOTE: this feature is still under heavy development. If you use it you should expect changes to the API as well the on-disk structures until it finalized and optimized". WDYT?
   
   > If we propose adding this to the core functionality though, I'd like to further discuss the pros/cons of how specific we make this new doc values field.
   
   At this point I really think we can continue with BDV. Only if someone has prior knowledge about the data and can encode each column separately, would it (perhaps) make sense to consider a different encoding?
   
   We can also consider including a `VERSION` identifier on-disk which we'll use to tell how to read the data during aggregation. It can just be another "long" we add to the beginning of the list. Yes, it's less optimal than a dedicated DV type, since it's repeating the same number for every field in every document, but it's something we can consider if we worry about it. Frankly though, I feel like we can relax the requirement of this feature at the start and not worry about it until after we're ready to remove that `NOTE:` from the javadocs.


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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1142455129

   > As for creating a new multidimensional version of the `Facets` API, I think what you're saying makes sense, but could we extend the existing `Facets` API as well, I think those methods would still be relevant right?
   
   I don't know, we'll need to review it when we get there. But returning a matrix is different than returning top-children IMO, so not sure it's worth to try too hard to make the `Facets` API return N-domensional results?


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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r874371423


##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoublePointFacetField.java:
##########
@@ -0,0 +1,47 @@
+/*
+ * 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 org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * Takes an array of doubles and converts them to sortable longs, then stores as a {@link
+ * BinaryDocValuesField}
+ */
+public class DoublePointFacetField extends BinaryDocValuesField {
+
+  /**
+   * Creates a new DoublePointFacetField, indexing the provided N-dimensional long point.
+   *
+   * @param name field name
+   * @param point double[] value
+   * @throws IllegalArgumentException if the field name or value is null.
+   */
+  public DoublePointFacetField(String name, double... point) {
+    super(name, LongPoint.pack(convertToSortableLongPoint(point)));
+  }
+
+  private static long[] convertToSortableLongPoint(double[] point) {
+    long[] ret = new long[point.length];

Review Comment:
   nit: I think this can be written w/ `Stream`, since it's called in the ctor of the Field I don't think we should worry about perf. Something like: `Arrays.stream(point).mapToObject(NumericUtils::doubleToSortableLong).toArray();`. Up to you though :)



##########
lucene/core/src/java/org/apache/lucene/document/LongPoint.java:
##########
@@ -117,6 +117,25 @@ public static BytesRef pack(long... point) {
     return new BytesRef(packed);
   }
 
+  /**
+   * Unpack a BytesRef into a long point
+   *
+   * @param bytesRef BytesRef Value
+   * @throws IllegalArgumentException the value is null
+   */
+  public static long[] unpack(BytesRef bytesRef) {

Review Comment:
   nit: can you please think about removing this method in favor of reversing the loops order in the collector? If we can do that, then we don't need to unpack to a `long[]` at all, but rather iterate on the dims (in the collector) and call `decodeDimension` there directly, each time evaluating a single `point` against all given `rectangles`. Will that work?



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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 org.apache.lucene.util.NumericUtils;
+
+/** Stores a hyper rectangle as an array of DoubleRangePairs */
+public class DoubleHyperRectangle extends HyperRectangle {
+
+  /** Stores pair as LongRangePair */
+  private final LongHyperRectangle.LongRangePair[] pairs;
+
+  /** Created DoubleHyperRectangle */
+  public DoubleHyperRectangle(String label, DoubleRangePair... pairs) {
+    super(label, checkPairsAndGetDim(pairs));
+    this.pairs = new LongHyperRectangle.LongRangePair[pairs.length];
+    for (int dim = 0; dim < pairs.length; dim++) {
+      long longMin = NumericUtils.doubleToSortableLong(pairs[dim].min);
+      long longMax = NumericUtils.doubleToSortableLong(pairs[dim].max);
+      this.pairs[dim] = new LongHyperRectangle.LongRangePair(longMin, true, longMax, true);

Review Comment:
   Is it correct to always pass `true` (inclusive)? I'm thinking perhaps we should introduce a `toLongRangePair` on `DoubleRangePair` which will (1) simplify this code and (2) use the actual values of inclusive for min/max? WDYT?



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangle.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+/** Holds the name and the number of dims for a HyperRectangle */
+public abstract class HyperRectangle {
+  /** Label that identifies this range. */
+  public final String label;
+
+  /** How many dimensions this hyper rectangle has (IE: a regular rectangle would have dims=2) */
+  public final int dims;
+
+  /** Sole constructor. */
+  protected HyperRectangle(String label, int dims) {
+    if (label == null) {
+      throw new IllegalArgumentException("label must not be null");
+    }
+    if (dims <= 0) {
+      throw new IllegalArgumentException("Dims must be greater than 0. Dims=" + dims);
+    }
+    this.label = label;
+    this.dims = dims;
+  }
+
+  /**
+   * Converts hyper rectangles ranges into a comparable long from whatever type it is in
+   *
+   * @param dim dimension of the request range
+   * @return The comparable long version of the requested range
+   */
+  public abstract LongHyperRectangle.LongRangePair getComparableDimRange(int dim);

Review Comment:
   After your refactoring, it seems that the two variants of this class don't "convert" anything here, but rather just do `return pairs[dim]`, so I wonder if we should rename this method / update the javadocs to remove "conversion" from it, and/or store a `LongRangePair[]` in this abstract class and have it take them in the constructor. Then we can implement this API in one place only.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.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 by 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)

Review Comment:
   I bet this is a remnant from old refactoring :), but I'll go with consistency too.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    this.field = field;
+    this.hyperRectangles = hyperRectangles;
+    this.dims = hyperRectangles[0].dims;
+    assert isHyperRectangleDimsConsistent()
+        : "All hyper rectangles must be the same dimensionality";
+    this.counts = new int[hyperRectangles.length];
+    count(field, hits.getMatchingDocs());
+  }
+
+  private boolean isHyperRectangleDimsConsistent() {
+    for (HyperRectangle hyperRectangle : hyperRectangles) {
+      if (hyperRectangle.dims != this.dims) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /** 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; ) {
+        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++) {
+              LongHyperRectangle.LongRangePair range =
+                  hyperRectangles[j].getComparableDimRange(dim);
+              if (!range.accept(point[dim])) {
+                validPoint = false;
+                break;
+              }
+            }
+            if (validPoint) {
+              counts[j]++;
+              docIsValid = true;
+            }
+          }
+          if (docIsValid) {
+            totCount++;
+          }
+        }
+        doc = it.nextDoc();
+      }
+    }
+  }
+
+  @Override
+  public FacetResult getTopChildren(int topN, String dim, String... path) throws IOException {
+    validateTopN(topN);
+    if (dim.equals(field) == false) {

Review Comment:
   nit: if you revert the `equals` check to `field.equals(dim)` then you don't risk NPE in case someone passes a null `dim` (and `field` is asserted in the ctor).



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java:
##########
@@ -0,0 +1,90 @@
+/*
+ * 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 org.apache.lucene.util.NumericUtils;
+
+/** Stores a hyper rectangle as an array of DoubleRangePairs */
+public class DoubleHyperRectangle extends HyperRectangle {
+
+  /** Stores pair as LongRangePair */
+  private final LongHyperRectangle.LongRangePair[] pairs;
+
+  /** Created DoubleHyperRectangle */

Review Comment:
   Creates?



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    this.field = field;
+    this.hyperRectangles = hyperRectangles;
+    this.dims = hyperRectangles[0].dims;
+    assert isHyperRectangleDimsConsistent()

Review Comment:
   I prefer that we do the assertions first, and only then assign to class fields. You can just pass the rectangles + dims to this method



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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.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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    this.field = field;
+    this.hyperRectangles = hyperRectangles;
+    this.dims = hyperRectangles[0].dims;
+    assert isHyperRectangleDimsConsistent()
+        : "All hyper rectangles must be the same dimensionality";
+    this.counts = new int[hyperRectangles.length];
+    count(field, hits.getMatchingDocs());
+  }
+
+  private boolean isHyperRectangleDimsConsistent() {
+    for (HyperRectangle hyperRectangle : hyperRectangles) {

Review Comment:
   I am not sure how do you feel about using `Stream` but if you're OK with it this one can just be written as `return Arrays.stream(hyperRectangles).allMatch(hyperRectangle -> hyperRectangle.dims == dims)`



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangle.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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;
+
+/** Holds the name and the number of dims for a HyperRectangle */
+public abstract class HyperRectangle {
+  /** Label that identifies this range. */
+  public final String label;
+
+  /** How many dimensions this hyper rectangle has (IE: a regular rectangle would have dims=2) */
+  public final int dims;
+
+  /** Sole constructor. */
+  protected HyperRectangle(String label, int dims) {
+    if (label == null) {
+      throw new IllegalArgumentException("label must not be null");
+    }
+    if (dims <= 0) {
+      throw new IllegalArgumentException("Dims must be greater than 0. Dims=" + dims);
+    }
+    this.label = label;
+    this.dims = dims;
+  }
+
+  /**
+   * Converts hyper rectangles ranges into a comparable long from whatever type it is in
+   *
+   * @param dim dimension of the request range
+   * @return The comparable long version of the requested range
+   */
+  public abstract LongHyperRectangle.LongRangePair getComparableDimRange(int dim);

Review Comment:
   If you accept that, you can also move `LongRangePair` here, which will make the code less odd -- DoubleRangePair and the Collector referencing a type from `LongHyperRectangle` even though they interact w/ `HyperRectangle`



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.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 by 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)

Review Comment:
   What prevents you from having one public ctor which just takes `HyperRectangle...` parameter?



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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r876602835


##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    assert isHyperRectangleDimsConsistent(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 isHyperRectangleDimsConsistent(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; ) {
+        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++;
+          }
+        }
+        doc = it.nextDoc();
+      }
+    }
+  }
+
+  @Override
+  public FacetResult getTopChildren(int topN, String dim, String... path) throws IOException {
+    validateTopN(topN);
+    if (field.equals(dim) == false) {
+      throw new IllegalArgumentException(
+          "invalid dim \"" + dim + "\"; should be \"" + field + "\"");
+    }
+    if (path.length != 0) {
+      throw new IllegalArgumentException("path.length should be 0");

Review Comment:
   Ugh you're right, I did get it backwards 🤦



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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1141152098

   Hey @mdmarshmallow I think this is a great and very useful feature. I also believe that in general it will be good to accompany these changes with a demo `main()` in the `demo` package, but it can wait a bit until we have a solid API.  I've added to this PR an `.adoc` with few example use cases. IMO it will be useful to keep it around, but modify it of course per the feedback we receive, as a documentation of this feature. If for some reason we'd think that this document is redundant / will be hard to maintain and we'll want to stick with javadocs, I don't mind if in the end we'll delete it. For now I think it's a convenient place to document our thoughts, examples and APIs.
   
   I used the term `FacetSets` to denote "a set of values that go together". Other names may include `Tuple`, `Group` etc. I know naming is the hardest part :). In my mind I'm also thinking about an API like:
   
   ```
   doc.add(new FacetSetsField(
       "actorAwards,
       // A Thriller for which this actor received a Best Actor Oscar award in 2022
       new FacetSet(ord("Oscar"), ord("Best Actor"), ord("Thriller"), 2022),
       // A Drama for which this actor received a Best Supporting Actor Emmy award in 2005
       new FacetSet(ord("Emmy"), ord("Best Supporting Actor"), ord("Drama"), 2005),
       ));
   ```
   
   Yes, it could be just sugar API on top of `HyperRectangle` but perhaps from a faceting perspective might make more sense and consistent with the other faceting API (`RangeFacets`, `SSDVFacetField` etc.). I'd love to receive feedback on the use cases. I can also add to the document a more-than-pseudocode-like example which will include the indexing and aggregation API, so we have something more concrete to discuss?


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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1148199584

   > I think the biggest difference between our implementations is the naming scheme.
   
   That's right. We could get both APIs to the same state of extensibility, but the main difference is the naming. Under the hood it's about encoding a series of `long[]` and implementing some aggregation function over them.
   
   > Maybe something like `RangedAndExactFacetSetMatcher` that lets user specify which which dimensions they want as exact matches, the details don't matter too much right now.
   
   We could totally offer that out-of-the-box, but I prefer that we do so in a different PR. Not sure about the API details yet, and whether it can truly be a generic impl or not, but it's totally something we should think about.
   
   > but I still feel like overriding the `matches` function is still a bit of an expert use case and no super clear maybe?
   
   Indeed, implementing your own `FacetSetMatcher` is expert? I mean, OOTB we offer few generic implementations which you can use to pretty much implement your facet set aggregation. If however you require a different _matching_ impl, or more specialized one, you can implement your own `FacetSetMatcher`. Also, given the OOTB examples, one should easily be able to understand how to implement their own.
   
   > So instead of letting the user override `match()`,
   
   Just so I understand and clarify that we're talking about the same thing: there are two classes here - `MatchingFacetSetCounts` (MFSC) and `FacetSetMatcher` (FSM). I think that MFSC can be `final` since I don't expect users to extend it. All they need is to provide a list of FSMs. Extending FSM is what you refer to here? This is indeed the more _expert_ API, which I hope users won't have to extend, given the OOTB classes.
   
   > we can create an `abstract readToLong()` that would read the field that the users created and stored
   
   So I wanted to allow FSM impls to be as optimal as they need, hence they are given a `byte[]`. We could consider passing `long[]` in the API, but that would mean that MFSC needs to deserialize the bytes to longs, which is what we wanted to avoid in the HyperRectangle impl. For this reason I don't think FSM should make you read the bytes into longs, let the impl do that if it's more convenient for it. Maybe we'll even find out that deserializing them to `long[]` (or reading them one `long` at a time) performs better, especially if the same dimension is evaluated over and over in a single `match()` call?
   
   I would also like to add that this sort of API is always something we can add in the future, in the form of `LongFSM` which overrides `match()` and adds its own `abstract long readLong(...)` version. That is, I'm not sure we need to finalize the details of this API just yet?
   
   > What if we made that `abstract FacetSet<T>` and provide an `abstract long[] writeToLong(T... values)`.
   
   Do you see more than two impls here? I.e. I put `FacetSet` but I wrote that we should also have `DoubleFacetSet` which encodes doubles. Any other impls here? Also, I'm not sure we should offer this extension to users, they can always implement that themselves? If you think about it, we eventually utilize BinaryDV to encode some data and aggregate it. The indexing + reading part of the code is not that complex so users can just write their own, and reuse existing classes where they fit?
   
   If users can decide how to write their `long[]`, they will also need to implement the reader, which in this PR is MFSC? If we'll change the aggregation API to be `long`-based then maybe? I think though that we may not need the generic here anyway. `FacetSet` can store `long[]` with a default impl for `toBytes[]` and you can override it if you wish. But this is super-expert API IMO, that I think we should expose only later if the need arises.


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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1149554008

   > `facetset` for the exact implementation and `hyperrectangle` for the range implementation
   
   If I understand correctly, @gsmiller's proposal was more about efficiency, right? So if you have a large number of points that you'd like to index, an R-Tree might be a better data structure for these range matches, than the current linear scan we do in `RangeFSM`. But it doesn't mean we shouldn't have `RangeFSM` for a small dimensionality / small number of FacetSet per doc cases right? They might even perform better?
   
   IMO, and to echo Greg's "progress not perfection", I feel it's better if we introduce the `HyperRectangle` impls (1) when we have a good use case to demonstrate the need for it (it just helps to reason about it) and (2) when we're actually ready to implement it "more efficiently". That is, in this PR I feel like we can do w/ `RangeFSM` and later (if and when we'll have a better alternative for large dims) document and reference the alternative?
   
   > For the second question, I think we should keep this as a `long[]` based API as we know we want to make the KD tree and R tree optimizations in the future
   
   If we think that KD-Trees can be more efficient for `FacetSetMatcher` impls and we're sure that we'll get there _soon_, then yeah, we can move to `long[]` based API. Another alternative I thought of is keep the `byte[]` API, but introduce a helper method on `FSM` which converts the `byte[]` to `long[]` for whoever needs it.
   
   To re-iterate what I previously wrote about API back-compat, I feel that putting `@lucene.experimental` tag on the classes is enough to have us not worry about changing it. Especially if the API hasn't been released yet, and definitely if we intend to follow with a KD-Tree impl very soon. A year from now (just an example) it's a different story, but for now I don't think we need to finalize the API for good.


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1150439971

   Let me try to summarize my understanding of the "future optimization" debate as it pertains to this proposed API/implementation and see if we're on the same page or if I'm overlooking something.
   
   The current proposal encapsulates/delegates matching logic behind `FacetSetMatcher`, which is responsible for determining whether-or-not that `FSM` instance matches a provided point. There's `RangeFSM` which knows how to match based on whether-or-not the point is contained in the n-dim range, and there's `ExactFSM` which is just doing exact point equivalence. The "protocol" for doing facet aggregation/counting is implemented inside `MatchingFacetSetsCounts`, which delegates to the `FSM#matches` method. The inefficiency is that—because `MatchingFacetSetsCounts` can make no assumptions about the `FSM` instances provided to it, it must iterate _all_ provided `FSM` instances for every indexed point for every provided hit.
   
   For the single point case (`ExactFSM`), this is crazy right? Even if there are a small number of provided `ExactFSM` instances we're matching against, doing a linear scan of all of them for every point is pretty dang inefficient. Especially so for a case where there are many provided hits with many indexed points for each. I think the same logic generally holds true for the range case as well, but maybe that's more debatable.
   
   But, the problem is, `MatchingFacetSetsCounts` doesn't know anything about these `FSM` instances it gets and can't do anything other than just ask them all, "hey, do you match this point?" And so the debate seems to be how to setup the API to allow for future optimizations, or if we should even worry about it at all.
   
   I personally think we should design with this optimization in mind, but I think we're close and I don't actually think the current proposal needs to really change to allow for future optimizations.
   
   This is where I get a little fuzzy on the conversation that's taken place as I haven't totally caught up on the various proposals, and conversations taking place. But, if we kept the implementation as it currently exists, in the future, if we want to put these optimizations in place, could we not just add a method to `FSM` that exposes the min/max values for each dimension? Then, `MatchingFacetSetsCounts` could inspect the underlying "hyperrectangles" represented by each `FSM` by looking at these min/max values before it counts and decide how it wants to proceed. The `matches` method is potentially still useful/needed depending on how flexible we want this thing to be; if a user creates an `FSM` implementation that's more complicated than just a "hyperrectangle" (e.g., some complicated geometry), the contract for providing min/max could be that the provided min/max is a bounding box, but `matches` still needs to be called to actually confirm the match.
   
   I point this out not to propose implementing it now, but to say that I think we have options to extend what is currently proposed here if/when we want to optimize. Does this make sense or am I talking about a completely different problem or missing key parts of the conversation that's happened? Apologies if I have.


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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r897533011


##########
lucene/facet/docs/FacetSets.adoc:
##########
@@ -0,0 +1,90 @@
+= FacetSets Overview
+:toc:
+
+This document describes the `FacetSets` capability, which allows to aggregate on multi dimensional values. It starts
+with outlining a few example use cases to showcase the motivation for this capability and follows with an API
+walk through.
+
+== Motivation
+
+[#movie-actors]
+=== Movie Actors DB
+
+Suppose that you want to build a search engine for movie actors which allows you to search for actors by name and see
+movie titles they appeared in. You might want to index standard fields such as `actorName`, `genre` and `releaseYear`
+which will let you search by the actor's name or see all actors who appeared in movies during 2021. Similarly, you can
+index facet fields that will let you aggregate by “Genre” and “Year” so that you can show how many actors appeared in
+each year or genre. Few example documents:
+
+[source]
+----
+{ "name": "Tom Hanks", "genre": ["Comedy", "Drama", …], "year": [1988, 2000,…] }
+{ "name": "Harrison Ford", "genre": ["Action", "Adventure", …], "year": [1977, 1981, …] }
+----
+
+However, these facet fields do not allow you to show the following aggregation:
+
+.Number of Actors performing in movies by Genre and Year
+[cols="4*"]
+|===
+| | 2020 | 2021 | 2022
+| Thriller | 121 | 43 | 97
+| Action | 145 | 52 | 130
+| Adventure | 87 | 21 | 32
+|===
+
+The reason is that each “genre” or “releaseYear” facet field is indexed in its own data structure, and therefore if an
+actor appeared in a "Thriller" movie in "2020" and "Action" movie in "2021", there's no way for you to tell that they
+didn't appear in an "Action" movie in "2020".
+
+[#automotive-parts]
+=== Automotive Parts Store
+
+Say you're building a search engine for an automotive parts store where customers can search for different car parts.
+For simplicity let's assume that each item in the catalog contains a searchable “type” field and “car model” it fits
+which consists of two separate fields: “manufacturer” and “year”. This lets you search for parts by their type as well
+as filter parts that fit only a certain manufacturer or year. Few example documents:
+
+[source]
+----
+{
+  "type": "Wiper Blades V1",
+  "models": [
+    { "manufaturer": "Ford", "year": 2010 },
+    { "manufacturer": "Chevy", "year": 2011 }
+  ]
+}
+{
+  "type": "Wiper Blades V2",
+  "models": [
+    { "manufaturer": "Ford", "year": 2011 },
+    { "manufacturer": "Chevy", "year": 2010 }
+  ]
+}
+----
+
+By breaking up the "models" field into its sub-fields "manufacturer" and "year", you can easily aggregate on parts that
+fit a certain manufacturer or year. However, if a user would like to aggregate on parts that can fit either a "Ford
+2010" or "Chevy 2011", then aggregating on the sub-fields will lead to a wrong count of 2 (in the above example) instead
+of 1.
+
+[#movie-awards]
+=== Movie Awards
+
+To showcase a 3-D multi-dimensional aggregation, lets expand the <<movie-actors>> example with awards an actor has
+received over the years. For this aggregation we will use four dimensions: Award Type ("Oscar", "Grammy", "Emmy"),
+Award Category ("Best Actor", "Best Supporting Actress"), Year and Genre. One interesting aggregation is to show how
+many "Best Actor" vs "Best Supporting Actor" awards one has received in the "Oscar" or "Emmy" for each year. Another
+aggregation is slicing the number of these awards by Genre over all the years.
+
+Building on these examples, one might be able to come up with an interesting use case for an N-dimensional aggregation
+(where `N > 3`). The higher `N` is, the harder it is to aggregate all the dimensions correctly and efficiently without
+`FacetSets`.
+
+== FacetSets API
+
+TBD
+
+== FacetSets Under the Hood
+
+TBD

Review Comment:
   I intended to do that, just wanted us to finalize the API first.



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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r898280617


##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSetsField.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.facetset;
+
+import java.util.Arrays;
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A {@link BinaryDocValuesField} which encodes a list of {@link FacetSet facet sets}. The encoding
+ * scheme consists of a packed {@code byte[]} where the first value denotes the number of dimensions
+ * in all the sets, followed by each set's values.
+ *
+ * @lucene.experimental
+ */
+public class FacetSetsField extends BinaryDocValuesField {
+
+  /**
+   * Create a new FacetSets field.
+   *
+   * @param name field name
+   * @param facetSets the {@link FacetSet facet sets} to index in that field. All must have the same
+   *     number of dimensions
+   * @throws IllegalArgumentException if the field name is null or the given facet sets are invalid
+   */
+  public static FacetSetsField create(String name, FacetSet... facetSets) {
+    if (facetSets == null || facetSets.length == 0) {
+      throw new IllegalArgumentException("FacetSets cannot be null or empty!");
+    }
+
+    return new FacetSetsField(name, toPackedValues(facetSets));
+  }
+
+  private FacetSetsField(String name, BytesRef value) {
+    super(name, value);
+  }
+
+  private static BytesRef toPackedValues(FacetSet... facetSets) {
+    int numDims = facetSets[0].dims;
+    // We could have created a buffer that can accommodate Long.BYTES per dimension value in each
+    // facet set. The below attempts to avoid allocating unnecessarily bigger arrays.
+    byte[] buf = new byte[4 + Arrays.stream(facetSets).mapToInt(FacetSet::sizePackedBytes).sum()];

Review Comment:
   Yup good idea



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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1156829025

   > I think we should also have some subpackages (maybe a `decoders`, `matchers`, and `sets` (for `FacetSets`))?
   
   Take a look at the last commit: not sure there's a reason for `decoders` at this point, since I've only added the interface w/ two static impls `decodeLongs` and `decodeInts`. Also `matchers` will hold 3 classes? Seems a bit overkill to me, WDYT?


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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1156833064

   @gsmiller @mdmarshmallow in the last commit I refactored `FacetSet` to be an abstract class with a concrete `LongFacetSet` impl. I struggled back-and-forth between introducing a `public final long[] comparableLongs` on the abstract `FacetSet` to the `getComparableLongs()` method. I went with the latter approach. Thoughts? Preferences?
   
   Also since I want to expose the raw values in each `FS`, having `DoubleFS` extend `LongFS` is not possible without exposing two fields from `DoubleFS` which I don't think that we want.
   
   Anyway I'm open to suggestions :). I also think it will be good to demonstrate somewhere, in the `.adoc`, `demo` package or tests how one can implement a specific mix-and-match `FS`, e.g. one that takes a long, int and float dimension values (we should build a story around that) together with a matching `FacetSetDecoder`. This will also prove that it's doable and that we won't break the API in ways that will prevent such impls.


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


[GitHub] [lucene] mdmarshmallow commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1142403496

   There was an email thread where some other commiters suggested also developing this in sandbox. It does seem like this API could go through some heavy changes (I think we all agree on that here), so it seems like the `sandbox` module would make more sense for this? Is there a benefit to having it in `facets` vs `sandbox`? I think putting this in a field that extends `BDV` makes the most sense right now as well. I think the `VERSION` identifier might be a bit overkill though, especially if we decide to put this in sandbox. I don't think we should worry about making this backwards compatible.
   
   As for creating a new multidimensional version of the `Facets` API, I think what you're saying makes sense, but could we extend the existing `Facets` API as well, I think those methods would still be relevant right?


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


[GitHub] [lucene] mdmarshmallow commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1148871572

   Hey Greg, I had a question about point 4. Are you saying we should have a separate hyper rectangle implementation in addition to facet sets in order to implement the R-tree and KD-tree optimizations? I actually addressed this above but I think we can just implement those in facet sets (specifically `MatchingFacetSetCounts`) right? Couldn't we have the class do put `RangeFSM` into R-trees and `ExactFSM` into KD-trees and that would fix the issue you are talking about here? I could be misunderstanding something so let me know.


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r877627338


##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.util.Arrays;
+import org.apache.lucene.util.NumericUtils;
+
+/** Stores a hyper rectangle as an array of DoubleRangePairs */
+public class DoubleHyperRectangle extends HyperRectangle {

Review Comment:
   As sort of a blanket comment for all the new classes you created, I might suggest we add `@lucene.experimental`. This is a pretty big, new bit of work, and I could see us wanting to tweak and change this a bit here-and-there.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.util.Arrays;
+import org.apache.lucene.util.NumericUtils;
+
+/** Stores a hyper rectangle as an array of DoubleRangePairs */
+public class DoubleHyperRectangle extends HyperRectangle {
+
+  /** Creates DoubleHyperRectangle */
+  public DoubleHyperRectangle(String label, DoubleRangePair... pairs) {
+    super(label, convertToLongRangePairs(pairs));
+  }
+
+  private static LongRangePair[] convertToLongRangePairs(DoubleRangePair... pairs) {
+    if (pairs == null || pairs.length == 0) {
+      throw new IllegalArgumentException("Pairs cannot be null or empty");
+    }
+    return Arrays.stream(pairs).map(DoubleRangePair::toLongRangePair).toArray(LongRangePair[]::new);
+  }
+
+  /** Defines a single range in a DoubleHyperRectangle */
+  public static class DoubleRangePair {
+    /** Inclusive min */
+    public final double min;
+
+    /** Inclusive max */
+    public final double max;
+
+    /**
+     * Creates a DoubleRangePair, very similar to the constructor of {@link
+     * org.apache.lucene.facet.range.DoubleRange}
+     *
+     * @param minIn Min value of pair
+     * @param minInclusive If minIn is inclusive
+     * @param maxIn Max value of pair
+     * @param maxInclusive If maxIn is inclusive
+     */
+    public DoubleRangePair(double minIn, boolean minInclusive, double maxIn, boolean maxInclusive) {
+      if (Double.isNaN(minIn) || Double.isNaN(maxIn)) {
+        throw new IllegalArgumentException(
+            "min and max cannot be NaN: min=" + minIn + ", max=" + maxIn);
+      }
+
+      if (!minInclusive) {
+        minIn = Math.nextUp(minIn);
+      }
+
+      if (!maxInclusive) {
+        maxIn = Math.nextDown(maxIn);
+      }
+
+      if (minIn > maxIn) {
+        throw new IllegalArgumentException(
+            "Minimum cannot be greater than maximum, max=" + maxIn + ", min=" + minIn);
+      }
+
+      this.min = minIn;
+      this.max = maxIn;
+    }
+
+    /**
+     * Converts this to a LongRangePair with sortable long equivalents
+     *
+     * @return A LongRangePair equivalent of this object
+     */
+    public LongRangePair toLongRangePair() {

Review Comment:
   Does this need to be public? I think it's only used internally in `DoubleHyperRectangle` right? Should we reduce visibility (unless we expect users need this functionality directly?).



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

Review Comment:
   I noticed you made all these fields `protected`. Were you thinking this might be a useful extension point for users? I might recommend against that, at least for now. If users start extending this, it might limit the changes we can make going forward (needing to stay back-compat with some of our internal implementation details).



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangle.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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;
+
+/** Holds the name and the number of dims for a HyperRectangle */
+public abstract class HyperRectangle {

Review Comment:
   Can this be made pkg-private?



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/LongPointFacetField.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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 org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.LongPoint;
+
+/** Packs an array of longs into a {@link BinaryDocValuesField} */
+public class LongPointFacetField extends BinaryDocValuesField {

Review Comment:
   I was thinking about these field classes you created a little more, and I'm wondering if we should create something more generic than just for faceting? I think what you've may run into here is the fact that we don't actually have a field type for indexing point values as doc values (that I know of anyway). We have all the `xxxPoint` fields for adding inverted fields in the points index (e.g., `LongPoint`), but I don't think we have an actual representation for adding them as DVs. 
   
   What do you think of moving these into the `document` package and actually defining them as general DV field types? We might not have to go so far as to actually formalize this new concept in the `DocValuesType` enum with their own format and such. Under the hood, they could just be a binary format like you have here (at least to start). You might look at `LongRangeDocValuesField` as a good example of what I mean.



##########
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. */

Review Comment:
   typo?



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


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

Posted by GitBox <gi...@apache.org>.
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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1154142817

   > Anyway, let's benchmark it, but with the analysis above, I also agree we should actually start with the long[] API, and replace it with a byte[] one only if actually performs better.
   
   +1 to starting with `long[]` and then benchmarking a `byte[]` version when time permits.
   
   > If I understand your change correctly, then it creates a new long[] in each call to matches() right? I see two main problems here
   
   Yeah, good callouts. I put this together pretty quickly as a sketched out idea, and didn't think super deeply about it. I was going for an approach that would let users extend the long-based API as the common approach, but allow extending the byte-based API if they really care about performance (but maybe it's not even more performant... TBD!).
   
   At this point, I'm convinced we should go with the long-based API for the initial version. Let's get this functionality shipped  and then we can benchmark, optimize, etc.


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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1153078375

   > pushed a change just now that shows a different way we _could_ approach this.
   
   If I understand your change correctly, then it creates a new `long[]` in each call to `matches()` right? I see two main problems here:
   
   1. It will potentially create millions of these small transient arrays if there are many hits and few facet sets per her.
   2. If there's more than one `FSM` then each one will decode the `byte[]` itself, right?
   
   Do you see any advantage of this impl over the one I pushed?
   
   BTW, we can remove the `numDims` parameter from my `matches(long[])` version, since `numDims == dimValues.length`, so it's redundant.


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r897216959


##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/ExactFacetSetMatcher.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.facetset;
+
+/**
+ * A {@link FacetSetMatcher} which considers a set as a match only if all dimension values are equal
+ * to the given one.
+ *
+ * @lucene.experimental
+ */
+public class ExactFacetSetMatcher extends FacetSetMatcher {

Review Comment:
   Should we include `Long` as part of the naming scheme for this (and `RangeFacetSetMatcher`) to note that it expects long points? I imagine we may want to create a "double" version of this in the future as well. Since we have different point types (`LongPoint`, `DoublePoint`, `IntPoint`, `FloatPoint`), we might need corresponding versions of these matchers for all of them right?



##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/RangeFacetSetMatcher.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.facetset;
+
+import java.util.Arrays;
+
+/**
+ * A {@link FacetSetMatcher} which considers a set as a match if all dimensions fall within the
+ * given corresponding range.
+ *
+ * @lucene.experimental
+ */
+public class RangeFacetSetMatcher extends FacetSetMatcher {
+
+  private final long[] lowerRanges;
+  private final long[] upperRanges;
+
+  /**
+   * Constructs and instance to match facet sets with dimensions that fall within the given ranges.

Review Comment:
   typo: "an instance" not "and instance"



##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/MatchingFacetSetsCounts.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.facetset;
+
+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.ConjunctionUtils;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Returns the counts for each given {@link FacetSet}
+ *
+ * @lucene.experimental
+ */
+public class MatchingFacetSetsCounts extends Facets {
+
+  private final FacetSetMatcher[] facetSetMatchers;
+  private final int[] counts;
+  private final String field;
+  private final int totCount;
+
+  /**
+   * Constructs a new instance of matching facet set counts which calculates the countBytes for each
+   * given facet set matcher.
+   */
+  public MatchingFacetSetsCounts(
+      String field, FacetsCollector hits, FacetSetMatcher... facetSetMatchers) throws IOException {
+    if (facetSetMatchers == null || facetSetMatchers.length == 0) {
+      throw new IllegalArgumentException("facetSetMatchers cannot be null or empty");
+    }
+    if (areFacetSetMatcherDimensionsInconsistent(facetSetMatchers)) {
+      throw new IllegalArgumentException("All facet set matchers must be the same dimensionality");
+    }
+    this.field = field;
+    this.facetSetMatchers = facetSetMatchers;
+    this.counts = new int[facetSetMatchers.length];
+    this.totCount = count(field, hits.getMatchingDocs());
+  }
+
+  /** Counts from the provided field. */
+  private int count(String field, List<FacetsCollector.MatchingDocs> matchingDocs)
+      throws IOException {
+
+    int totCount = 0;
+    for (FacetsCollector.MatchingDocs hits : matchingDocs) {
+
+      BinaryDocValues binaryDocValues = DocValues.getBinary(hits.context.reader(), field);
+
+      final DocIdSetIterator it =
+          ConjunctionUtils.intersectIterators(Arrays.asList(hits.bits.iterator(), binaryDocValues));
+      if (it == null) {
+        continue;
+      }
+
+      long[] dimValues = null; // dimension values buffer
+      int expectedNumDims = -1;
+      for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+        boolean shouldCountDoc = false;
+        BytesRef bytesRef = binaryDocValues.binaryValue();
+        byte[] packedValue = bytesRef.bytes;
+        int numDims = (int) LongPoint.decodeDimension(packedValue, 0);
+        if (expectedNumDims == -1) {
+          expectedNumDims = numDims;
+          dimValues = new long[numDims];
+        } else {
+          // Verify that the number of indexed dimensions for all matching documents is the same
+          // (since we cannot verify that at indexing time).
+          assert numDims == expectedNumDims
+              : "Expected ("
+                  + expectedNumDims
+                  + ") dimensions, found ("
+                  + numDims
+                  + ") for doc ("
+                  + doc
+                  + ")";
+        }
+
+        for (int start = Long.BYTES; start < bytesRef.length; start += numDims * Long.BYTES) {
+          LongPoint.unpack(bytesRef, start, dimValues);
+          for (int j = 0; j < facetSetMatchers.length; j++) { // for each facet set matcher
+            if (facetSetMatchers[j].matches(dimValues)) {
+              counts[j]++;
+              shouldCountDoc = true;
+            }
+          }
+        }
+        if (shouldCountDoc) {
+          totCount++;
+        }
+      }
+    }
+    return totCount;
+  }
+
+  // TODO: This does not really provide "top children" functionality yet but provides "all
+  // children". This is being worked on in LUCENE-10550
+  @Override
+  public FacetResult getTopChildren(int topN, String dim, String... path) throws IOException {
+    validateTopN(topN);
+    if (!field.equals(dim)) {

Review Comment:
   `== false` instead of `!` please



##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/MatchingFacetSetsCounts.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.facetset;
+
+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.ConjunctionUtils;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Returns the counts for each given {@link FacetSet}
+ *
+ * @lucene.experimental
+ */
+public class MatchingFacetSetsCounts extends Facets {

Review Comment:
   Does this also need to include `Long` in the name since it's specific to long-encoded points and long-based matchers? We'll need parallel versions of _all_ of this for different types if we want to support them right (double, int, float)?



##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/MatchingFacetSetsCounts.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.facetset;
+
+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.ConjunctionUtils;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Returns the counts for each given {@link FacetSet}

Review Comment:
   Update this javadoc to reference `FacetSetMatcher` and explain usage?



##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/RangeFacetSetMatcher.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.facetset;
+
+import java.util.Arrays;
+
+/**
+ * A {@link FacetSetMatcher} which considers a set as a match if all dimensions fall within the
+ * given corresponding range.
+ *
+ * @lucene.experimental
+ */
+public class RangeFacetSetMatcher extends FacetSetMatcher {
+
+  private final long[] lowerRanges;
+  private final long[] upperRanges;
+
+  /**
+   * Constructs and instance to match facet sets with dimensions that fall within the given ranges.
+   */
+  public RangeFacetSetMatcher(String label, LongRange... dimRanges) {
+    super(label, getDims(dimRanges));
+    this.lowerRanges = Arrays.stream(dimRanges).mapToLong(range -> range.min).toArray();
+    this.upperRanges = Arrays.stream(dimRanges).mapToLong(range -> range.max).toArray();
+  }
+
+  @Override
+  public boolean matches(long[] dimValues) {
+    assert dimValues.length == dims
+        : "Encoded dimensions (dims="
+            + dimValues.length
+            + ") is incompatible with range dimensions (dims="
+            + dims
+            + ")";
+
+    for (int i = 0; i < dimValues.length; i++) {
+      if (dimValues[i] < lowerRanges[i]) {
+        // Doc's value is too low in this dimension
+        return false;
+      }
+      if (dimValues[i] > upperRanges[i]) {
+        // Doc's value is too high in this dimension
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private static int getDims(LongRange... dimRanges) {
+    if (dimRanges == null || dimRanges.length == 0) {
+      throw new IllegalArgumentException("dimRanges cannot be null or empty");
+    }
+    return dimRanges.length;
+  }
+
+  /** Defines a single range in a FacetSet dimension. */
+  public static class LongRange {
+    /** Inclusive min */
+    public final long min;
+
+    /** Inclusive max */
+    public final long max;
+
+    /**
+     * Creates a LongRange.
+     *
+     * @param min min value in range
+     * @param minInclusive if min is inclusive
+     * @param max max value in range
+     * @param maxInclusive if max is inclusive
+     */
+    public LongRange(long min, boolean minInclusive, long max, boolean maxInclusive) {
+      if (!minInclusive) {
+        if (min != Long.MAX_VALUE) {
+          min++;
+        } else {
+          throw new IllegalArgumentException("Invalid min input: " + min);
+        }
+      }
+
+      if (!maxInclusive) {

Review Comment:
   minor: please use `== false` instead of `!`



##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/RangeFacetSetMatcher.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.facetset;
+
+import java.util.Arrays;
+
+/**
+ * A {@link FacetSetMatcher} which considers a set as a match if all dimensions fall within the
+ * given corresponding range.
+ *
+ * @lucene.experimental
+ */
+public class RangeFacetSetMatcher extends FacetSetMatcher {
+
+  private final long[] lowerRanges;
+  private final long[] upperRanges;
+
+  /**
+   * Constructs and instance to match facet sets with dimensions that fall within the given ranges.
+   */
+  public RangeFacetSetMatcher(String label, LongRange... dimRanges) {
+    super(label, getDims(dimRanges));
+    this.lowerRanges = Arrays.stream(dimRanges).mapToLong(range -> range.min).toArray();
+    this.upperRanges = Arrays.stream(dimRanges).mapToLong(range -> range.max).toArray();
+  }
+
+  @Override
+  public boolean matches(long[] dimValues) {
+    assert dimValues.length == dims
+        : "Encoded dimensions (dims="
+            + dimValues.length
+            + ") is incompatible with range dimensions (dims="
+            + dims
+            + ")";
+
+    for (int i = 0; i < dimValues.length; i++) {
+      if (dimValues[i] < lowerRanges[i]) {
+        // Doc's value is too low in this dimension
+        return false;
+      }
+      if (dimValues[i] > upperRanges[i]) {
+        // Doc's value is too high in this dimension
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private static int getDims(LongRange... dimRanges) {
+    if (dimRanges == null || dimRanges.length == 0) {
+      throw new IllegalArgumentException("dimRanges cannot be null or empty");
+    }
+    return dimRanges.length;
+  }
+
+  /** Defines a single range in a FacetSet dimension. */
+  public static class LongRange {
+    /** Inclusive min */
+    public final long min;
+
+    /** Inclusive max */
+    public final long max;
+
+    /**
+     * Creates a LongRange.
+     *
+     * @param min min value in range
+     * @param minInclusive if min is inclusive
+     * @param max max value in range
+     * @param maxInclusive if max is inclusive
+     */
+    public LongRange(long min, boolean minInclusive, long max, boolean maxInclusive) {
+      if (!minInclusive) {

Review Comment:
   minor: please use `== false` instead of `!`



##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/ExactFacetSetMatcher.java:
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.facetset;
+
+/**
+ * A {@link FacetSetMatcher} which considers a set as a match only if all dimension values are equal
+ * to the given one.
+ *
+ * @lucene.experimental
+ */
+public class ExactFacetSetMatcher extends FacetSetMatcher {
+
+  private final long[] values;
+
+  /** Constructs an instance to match the given facet set. */
+  public ExactFacetSetMatcher(String label, FacetSet facetSet) {
+    super(label, facetSet.values.length);
+    this.values = facetSet.values;
+  }
+
+  @Override
+  public boolean matches(long[] dimValues) {
+    assert dimValues.length == dims
+        : "Encoded dimensions (dims="
+            + dimValues.length
+            + ") is incompatible with FacetSet dimensions (dims="
+            + dims
+            + ")";
+
+    for (int i = 0; i < dimValues.length; i++) {
+      if (dimValues[i] != values[i]) {
+        // Field's dimension value is not equal to given dimension, the entire set is rejected
+        return false;
+      }
+    }
+    return true;

Review Comment:
   More idiomatic (and maybe more optimal) to do this instead?
   
   ```suggestion
   return Arrays.equals(dimValues, values);
   ```



##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSetsField.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.facetset;
+
+import java.util.Arrays;
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A {@link BinaryDocValuesField} which encodes a list of {@link FacetSet facet sets}. The encoding
+ * scheme consists of a packed {@code long[]} where the first value denotes the number of dimensions
+ * in all the sets, followed by each set's values.
+ *
+ * @lucene.experimental
+ */
+public class FacetSetsField extends BinaryDocValuesField {
+
+  /**
+   * Create a new FacetSets field.
+   *
+   * @param name field name
+   * @param facetSets the {@link FacetSet} to index in that field. All must have the same number of

Review Comment:
   minor: mention it's a _list_ of FacetSets?



##########
lucene/facet/docs/FacetSets.adoc:
##########
@@ -0,0 +1,90 @@
+= FacetSets Overview
+:toc:
+
+This document describes the `FacetSets` capability, which allows to aggregate on multi dimensional values. It starts
+with outlining a few example use cases to showcase the motivation for this capability and follows with an API
+walk through.
+
+== Motivation
+
+[#movie-actors]
+=== Movie Actors DB
+
+Suppose that you want to build a search engine for movie actors which allows you to search for actors by name and see
+movie titles they appeared in. You might want to index standard fields such as `actorName`, `genre` and `releaseYear`
+which will let you search by the actor's name or see all actors who appeared in movies during 2021. Similarly, you can
+index facet fields that will let you aggregate by “Genre” and “Year” so that you can show how many actors appeared in
+each year or genre. Few example documents:
+
+[source]
+----
+{ "name": "Tom Hanks", "genre": ["Comedy", "Drama", …], "year": [1988, 2000,…] }
+{ "name": "Harrison Ford", "genre": ["Action", "Adventure", …], "year": [1977, 1981, …] }
+----
+
+However, these facet fields do not allow you to show the following aggregation:
+
+.Number of Actors performing in movies by Genre and Year
+[cols="4*"]
+|===
+| | 2020 | 2021 | 2022
+| Thriller | 121 | 43 | 97
+| Action | 145 | 52 | 130
+| Adventure | 87 | 21 | 32
+|===
+
+The reason is that each “genre” or “releaseYear” facet field is indexed in its own data structure, and therefore if an
+actor appeared in a "Thriller" movie in "2020" and "Action" movie in "2021", there's no way for you to tell that they
+didn't appear in an "Action" movie in "2020".
+
+[#automotive-parts]
+=== Automotive Parts Store
+
+Say you're building a search engine for an automotive parts store where customers can search for different car parts.
+For simplicity let's assume that each item in the catalog contains a searchable “type” field and “car model” it fits
+which consists of two separate fields: “manufacturer” and “year”. This lets you search for parts by their type as well
+as filter parts that fit only a certain manufacturer or year. Few example documents:
+
+[source]
+----
+{
+  "type": "Wiper Blades V1",
+  "models": [
+    { "manufaturer": "Ford", "year": 2010 },
+    { "manufacturer": "Chevy", "year": 2011 }
+  ]
+}
+{
+  "type": "Wiper Blades V2",
+  "models": [
+    { "manufaturer": "Ford", "year": 2011 },
+    { "manufacturer": "Chevy", "year": 2010 }
+  ]
+}
+----
+
+By breaking up the "models" field into its sub-fields "manufacturer" and "year", you can easily aggregate on parts that
+fit a certain manufacturer or year. However, if a user would like to aggregate on parts that can fit either a "Ford
+2010" or "Chevy 2011", then aggregating on the sub-fields will lead to a wrong count of 2 (in the above example) instead
+of 1.
+
+[#movie-awards]
+=== Movie Awards
+
+To showcase a 3-D multi-dimensional aggregation, lets expand the <<movie-actors>> example with awards an actor has
+received over the years. For this aggregation we will use four dimensions: Award Type ("Oscar", "Grammy", "Emmy"),
+Award Category ("Best Actor", "Best Supporting Actress"), Year and Genre. One interesting aggregation is to show how
+many "Best Actor" vs "Best Supporting Actor" awards one has received in the "Oscar" or "Emmy" for each year. Another
+aggregation is slicing the number of these awards by Genre over all the years.
+
+Building on these examples, one might be able to come up with an interesting use case for an N-dimensional aggregation
+(where `N > 3`). The higher `N` is, the harder it is to aggregate all the dimensions correctly and efficiently without
+`FacetSets`.
+
+== FacetSets API
+
+TBD
+
+== FacetSets Under the Hood
+
+TBD

Review Comment:
   Fill in the TBDs?



##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/MatchingFacetSetsCounts.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.facetset;
+
+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.ConjunctionUtils;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Returns the counts for each given {@link FacetSet}
+ *
+ * @lucene.experimental
+ */
+public class MatchingFacetSetsCounts extends Facets {
+
+  private final FacetSetMatcher[] facetSetMatchers;
+  private final int[] counts;
+  private final String field;
+  private final int totCount;
+
+  /**
+   * Constructs a new instance of matching facet set counts which calculates the countBytes for each

Review Comment:
   What are `countBytes`?



##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSetsField.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.facetset;
+
+import java.util.Arrays;
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A {@link BinaryDocValuesField} which encodes a list of {@link FacetSet facet sets}. The encoding
+ * scheme consists of a packed {@code long[]} where the first value denotes the number of dimensions
+ * in all the sets, followed by each set's values.
+ *
+ * @lucene.experimental
+ */
+public class FacetSetsField extends BinaryDocValuesField {
+
+  /**
+   * Create a new FacetSets field.
+   *
+   * @param name field name
+   * @param facetSets the {@link FacetSet} to index in that field. All must have the same number of
+   *     dimensions
+   * @throws IllegalArgumentException if the field name is null or the given facet sets are invalid
+   */
+  public static FacetSetsField create(String name, FacetSet... facetSets) {
+    validateFacetSets(facetSets);
+
+    return new FacetSetsField(name, toPackedLongs(facetSets));
+  }
+
+  private FacetSetsField(String name, BytesRef value) {
+    super(name, value);
+  }
+
+  private static void validateFacetSets(FacetSet... facetSets) {

Review Comment:
   What about doing the validation as part of `toPackedLongs` instead? This is very minor, but it _would_ allow us to not iterate the facet sets twice to validate dimensions (we could validate dims as we pack in the `toPackedLongs` loop).



##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSetsField.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.facetset;
+
+import java.util.Arrays;
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A {@link BinaryDocValuesField} which encodes a list of {@link FacetSet facet sets}. The encoding
+ * scheme consists of a packed {@code long[]} where the first value denotes the number of dimensions
+ * in all the sets, followed by each set's values.
+ *
+ * @lucene.experimental
+ */
+public class FacetSetsField extends BinaryDocValuesField {
+
+  /**
+   * Create a new FacetSets field.
+   *
+   * @param name field name
+   * @param facetSets the {@link FacetSet} to index in that field. All must have the same number of
+   *     dimensions
+   * @throws IllegalArgumentException if the field name is null or the given facet sets are invalid
+   */
+  public static FacetSetsField create(String name, FacetSet... facetSets) {
+    validateFacetSets(facetSets);
+
+    return new FacetSetsField(name, toPackedLongs(facetSets));
+  }
+
+  private FacetSetsField(String name, BytesRef value) {
+    super(name, value);
+  }
+
+  private static void validateFacetSets(FacetSet... facetSets) {
+    if (facetSets == null || facetSets.length == 0) {
+      throw new IllegalArgumentException("FacetSets cannot be null or empty!");
+    }
+
+    int dims = facetSets[0].values.length;
+    if (!Arrays.stream(facetSets).allMatch(facetSet -> facetSet.values.length == dims)) {

Review Comment:
   minor: prefer to use `== false` instead of `!` in the codebase



##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/RangeFacetSetMatcher.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.facetset;
+
+import java.util.Arrays;
+
+/**
+ * A {@link FacetSetMatcher} which considers a set as a match if all dimensions fall within the
+ * given corresponding range.
+ *
+ * @lucene.experimental
+ */
+public class RangeFacetSetMatcher extends FacetSetMatcher {
+
+  private final long[] lowerRanges;
+  private final long[] upperRanges;
+
+  /**
+   * Constructs and instance to match facet sets with dimensions that fall within the given ranges.
+   */
+  public RangeFacetSetMatcher(String label, LongRange... dimRanges) {
+    super(label, getDims(dimRanges));
+    this.lowerRanges = Arrays.stream(dimRanges).mapToLong(range -> range.min).toArray();
+    this.upperRanges = Arrays.stream(dimRanges).mapToLong(range -> range.max).toArray();

Review Comment:
   I realize that optimizing this is likely unimportant since it's a one-time setup cost, but there's something about iterating the same data structure twice in a row that irks me. Could we do a traditional loop here and only do a single pass?



##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/MatchingFacetSetsCounts.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.facetset;
+
+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.ConjunctionUtils;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Returns the counts for each given {@link FacetSet}
+ *
+ * @lucene.experimental
+ */
+public class MatchingFacetSetsCounts extends Facets {
+
+  private final FacetSetMatcher[] facetSetMatchers;
+  private final int[] counts;
+  private final String field;
+  private final int totCount;
+
+  /**
+   * Constructs a new instance of matching facet set counts which calculates the countBytes for each
+   * given facet set matcher.
+   */
+  public MatchingFacetSetsCounts(
+      String field, FacetsCollector hits, FacetSetMatcher... facetSetMatchers) throws IOException {
+    if (facetSetMatchers == null || facetSetMatchers.length == 0) {
+      throw new IllegalArgumentException("facetSetMatchers cannot be null or empty");
+    }
+    if (areFacetSetMatcherDimensionsInconsistent(facetSetMatchers)) {
+      throw new IllegalArgumentException("All facet set matchers must be the same dimensionality");
+    }
+    this.field = field;
+    this.facetSetMatchers = facetSetMatchers;
+    this.counts = new int[facetSetMatchers.length];
+    this.totCount = count(field, hits.getMatchingDocs());
+  }
+
+  /** Counts from the provided field. */
+  private int count(String field, List<FacetsCollector.MatchingDocs> matchingDocs)

Review Comment:
   It's a bit inconsistent with other facet counting implementations that you return a total count here (but populate `counts` as a function side-effect). Could we make this a void method and just populate everything directly?



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


[GitHub] [lucene] mdmarshmallow commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1156800419

   I took a look at the API and seeing if we could do something there with generics, but that overcomplicates the API haha. It looks good to me! Maybe one thing would be that we want to make it mandatory for a `FacetSet` to implement its own `decode()` function? Also +1 to making a `LongFacetSet`, I think it will make the API much more clear.


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


[GitHub] [lucene] mdmarshmallow commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1156990982

   I like the `getComparableLongs()` approach better as well.
   
   I also think a demo is a good idea, I can create one later today or tomorrow if you don't get around to it before then.
   
   Also another thing I realized is that `RangeFacetSetMatcher` is a bit awkward now, as it actually is only specific to `long`'s. I added the ability to use `FacetSet`'s to define ranges (as long as they implement the `RangeMatching` interface). Let me know what you guys think of the changes. There is also an example of what the new `RangeFacetSetMatcher` interface will look like in the tests.


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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r903277452


##########
lucene/demo/src/java/org/apache/lucene/demo/facet/CustomFacetSetExample.java:
##########
@@ -0,0 +1,303 @@
+/*
+ * 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.time.LocalDate;
+import java.time.ZoneOffset;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.document.*;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollectorManager;
+import org.apache.lucene.facet.facetset.*;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.store.ByteBuffersDirectory;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * Shows usage of indexing and searching {@link FacetSetsField} with a custom {@link FacetSet}
+ * implementation. Unlike the out of the box {@link FacetSet} implementations, this example shows
+ * how to mix and match dimensions of different types, as well as implementing a custom {@link
+ * FacetSetMatcher}.
+ */
+public class CustomFacetSetExample {
+
+  private static final long MAY_SECOND_2022 = date("2022-05-02");
+  private static final long JUNE_SECOND_2022 = date("2022-06-02");
+  private static final long JULY_SECOND_2022 = date("2022-07-02");
+  private static final float HUNDRED_TWENTY_DEGREES = fahrenheitToCelsius(120);
+  private static final float HUNDRED_DEGREES = fahrenheitToCelsius(100);
+  private static final float EIGHTY_DEGREES = fahrenheitToCelsius(80);
+
+  private final Directory indexDir = new ByteBuffersDirectory();
+
+  /** Empty constructor */
+  public CustomFacetSetExample() {}
+
+  /** Build the example index. */
+  private void index() throws IOException {
+    IndexWriter indexWriter =
+        new IndexWriter(
+            indexDir, new IndexWriterConfig(new WhitespaceAnalyzer()).setOpenMode(OpenMode.CREATE));
+
+    // Every document holds the temperature measures for a City by Date
+
+    Document doc = new Document();
+    doc.add(new StringField("city", "city1", Field.Store.YES));
+    doc.add(
+        FacetSetsField.create(
+            "temperature",
+            new TemperatureReadingFacetSet(MAY_SECOND_2022, HUNDRED_DEGREES),
+            new TemperatureReadingFacetSet(JUNE_SECOND_2022, EIGHTY_DEGREES),
+            new TemperatureReadingFacetSet(JULY_SECOND_2022, HUNDRED_TWENTY_DEGREES)));
+    indexWriter.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new StringField("city", "city2", Field.Store.YES));
+    doc.add(
+        FacetSetsField.create(
+            "temperature",
+            new TemperatureReadingFacetSet(MAY_SECOND_2022, EIGHTY_DEGREES),
+            new TemperatureReadingFacetSet(JUNE_SECOND_2022, HUNDRED_DEGREES),
+            new TemperatureReadingFacetSet(JULY_SECOND_2022, HUNDRED_TWENTY_DEGREES)));
+    indexWriter.addDocument(doc);
+
+    indexWriter.close();
+  }
+
+  /** Counting documents which exactly match a given {@link FacetSet}. */
+  private List<FacetResult> exactMatching() throws IOException {
+    DirectoryReader indexReader = DirectoryReader.open(indexDir);
+    IndexSearcher searcher = new IndexSearcher(indexReader);
+
+    // MatchAllDocsQuery is for "browsing" (counts facets
+    // for all non-deleted docs in the index); normally
+    // you'd use a "normal" query:
+    FacetsCollector fc = searcher.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    // Count both "Publish Date" and "Author" dimensions

Review Comment:
   Indeed :), I copied the simple faceting example and didn't cover up my tracks very well :D.



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


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

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r881945764


##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/LongPointFacetField.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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 org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.LongPoint;
+
+/** Packs an array of longs into a {@link BinaryDocValuesField} */
+public class LongPointFacetField extends BinaryDocValuesField {

Review Comment:
   I was wondering what your thoughts were on just using separate numeric fields rather than packing them. I think this would make the API "nicer" to be honest, but the big drawback would that we would need some hacky multivalued implementation. I can think of some ways to build some sort of UnsortedNumericDV on top of SortedNumericDV, but they would all be super hacky and have limitations and probably not worth implementing.



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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1142433913

   > Is there a benefit to having it in `facets` vs `sandbox`
   
   I personally am not sure that we should worry about the big changes that the module will go under. API-wise we tag the classes as `@lucene.experimental` which should free us from worrying about the API for a while (but obviously at some point we'll want to declare the API stable!). As for the on-disk structure I also feel like we have some room with it. It's the `facet` module, not `core`. There are much less users of it and it's a completely new feature, so I think it's reasonable to declare the whole feature experimental.
   
   I worry that if we put it in sandbox, users might not even attempt to try it, even if technically they don't mind re-indexing on version upgrades. Because "sandbox" feels (to me) like half-baked stuff, while it's not true here - we do deliver value, it's just that the representation of things may change.


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1148846237

   OK, I've (somewhat) caught up on the conversation here and will follow up on my original questions/comments (but am not going to jump in right now on the latest API discussion).
   
   1. I like this "facet set" naming approach along with providing specific implementations for "exact match" cases and "range" cases. I think we should stick to these two for now. If a user wants to "mix and match" (some dims are exact matches and some are ranges), they can use the more general "range" implementation (with some dim ranges containing common values for min/max). Or they could of course implement their own. I don't think we need the complexity of an OOTB "mix and match" solution (for now at leat).
   2. As far as solving for use-cases where users want to "fix" the n-1 dims and then get top values for the nth dim, I don't think we need to solve for that (yet). The existing "range" facet counting doesn't solve for this, and requires users to fully describe the ranges they care about. So for the sake of "progress not perfection", I see no issue with following a similar pattern here.
   3. If users _do_ need to implement the above use-case (no. 2 above), there's actually a different way to go about it. Because `LongValueFacetCounts` allows users to provide a `LongValuesSource`, users can implement their own `LongValueSource` that provides values for the dimension they want to count, but pre-filters to only the points that match the n-1 filtering dims. So in the above example, if users wanted the top year values for movies that received the "Oscar+Drama" award, they can implement a `LongValuesSource` on top of the binary doc value field (the packed points) that "emits" the year value for each point, but only if it the other dims meet the "Oscar+Drama" criteria. I've actually done this in practice. We could certainly make this easier for users to do, but they have all the primitives to do this on their own (especially with the addition of the proposed `LongPointDocValuesField`).
   4. I think there's actually a nice future optimization that's a bit easier with modeling the "exact match" and "range" cases separately. If the user has many points or "hyperrectangles" specified, we might want to use some sort of space-partitioning data structure to make determining the matching points/hyperrectangles more efficient as we iterate the doc points (instead of doing an exhaustive search every time). These data structures will be different for these two cases (one is probably some sort of KD-tree for the "exact match" case and the other might be some sort of R-tree for the "hyperrectangle" case). So having these separate implementations might actually set us up for a nice performance improvement too, where if we modeled everything as "hyperrectangles", we could end up just stuffing a bunch of points into an R-tree which is a little weird.
   5. I look forward to seeing the "range" implementation sketched out :)


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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1151926798

   > So unless I'm totally missing something, I think we should keep the `matches(byte[])` method.
   
   I don't think you're missing something, but perhaps to give a different POV, I am thinking about users who will want to extend the API. As a user, it feels more intuitive to deal with a `long[]` than ` byte[]`. Take a look at the impls, they are much simpler and intuitive (IMO). It also _feels_ less expert and approachable.
   
   Another point which may change our minds about it -- when we give users a `byte[]`, we force them to know how the data was encoded. The two current `FSM`s need to iterate on the array with `Long.BYTES` increments. If we decided to encode the data in the BDV differently (maybe compress it), then a `byte[]` API will need to be accompanied with this information as well.
   
   So again, **purely from an API perspective**, we tell the user "You give us `long[]` at indexing time, we'll give it you back at aggregation time". It's simple, readable, intuitive.
   
   Perhaps we should include a comparison of these two variants in our benchmark? E.g. if there's no downside to using `long[]` then I assume we'll be fine with keeping it? But if it performs slower, than we can stick with the `byte[]` to keep things well optimized.


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1152648521

   > So again, purely from an API perspective, we tell the user "You give us long[] at indexing time, we'll give it you back at aggregation time". It's simple, readable, intuitive.
   
   Hmm, yeah this is a good point. +1 that it would be more intuitive for users to implement a long-based match method if they're writing their own `FSM` implementations. And +1 that it also provides a layer of separation between the encoding and the logic (so we could change the encoding of the point doc values without breaking `FSM`s).
   
   I pushed a change just now that shows a different way we _could_ approach this. I'm not sure I think we _should_, but wanted to float it out there to see what you all think. I'm also not opposed to moving forward with _only_ a long-based API and then benchmark to see if the byte-based approach is _actually_ more optimal. Honestly, that's probably the right decision here...


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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r901239793


##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/RangeFacetSetMatcher.java:
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.facetset;
+
+import java.util.Arrays;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * A {@link FacetSetMatcher} which considers a set as a match if all dimensions fall within the
+ * given corresponding range.
+ *
+ * @lucene.experimental
+ */
+public class RangeFacetSetMatcher extends FacetSetMatcher {
+
+  private final long[] lowerRanges;
+  private final long[] upperRanges;
+
+  /**
+   * Constructs an instance to match facet sets with dimensions that fall within the given ranges.
+   */
+  public RangeFacetSetMatcher(String label, DimRange... dimRanges) {
+    super(label, getDims(dimRanges));
+    this.lowerRanges = Arrays.stream(dimRanges).mapToLong(range -> range.min).toArray();
+    this.upperRanges = Arrays.stream(dimRanges).mapToLong(range -> range.max).toArray();
+  }
+
+  @Override
+  public boolean matches(long[] dimValues) {
+    assert dimValues.length == dims
+        : "Encoded dimensions (dims="
+            + dimValues.length
+            + ") is incompatible with range dimensions (dims="
+            + dims
+            + ")";
+
+    for (int i = 0; i < dimValues.length; i++) {
+      if (dimValues[i] < lowerRanges[i]) {
+        // Doc's value is too low in this dimension
+        return false;
+      }
+      if (dimValues[i] > upperRanges[i]) {
+        // Doc's value is too high in this dimension
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private static int getDims(DimRange... dimRanges) {
+    if (dimRanges == null || dimRanges.length == 0) {
+      throw new IllegalArgumentException("dimRanges cannot be null or empty");
+    }
+    return dimRanges.length;
+  }
+
+  /**
+   * Creates a {@link DimRange} for the given min and max long values. This method is also suitable
+   * for int values.
+   */
+  public static DimRange fromLongs(long min, boolean minInclusive, long max, boolean maxInclusive) {

Review Comment:
   Yeah makes sense to me too! The only bummer is that it makes lines such as `RangeFacetSetMatcher.fromLongs` become `RangeFacetSetMatcher.DimRange.fromLongs`. Should we extract `DimRange` as a top-level class? I'm not too obsessed about it though.



##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/package-info.java:
##########
@@ -0,0 +1,19 @@
+/*
+ * 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.
+ */
+
+/** Provides FacetSets faceting capabilities. */

Review Comment:
   Done



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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1163904822

   Thanks @mdmarshmallow. You added the CHANGES entry under `Lucene 9.30` so am just verifying -- we're going to merge it both to `main` and `9.x` branches?


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


[GitHub] [lucene] mdmarshmallow commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1163620698

   I think the rebase was somehow messed up, I cleaned up the history and force pushed. Everything should be included in this push.


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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r903278541


##########
lucene/facet/docs/FacetSets.adoc:
##########
@@ -0,0 +1,130 @@
+= FacetSets Overview
+:toc:
+
+This document describes the `FacetSets` capability, which allows to aggregate on multidimensional values. It starts
+with outlining a few example use cases to showcase the motivation for this capability and follows with an API
+walk through.
+
+== Motivation
+
+[#movie-actors]
+=== Movie Actors DB
+
+Suppose that you want to build a search engine for movie actors which allows you to search for actors by name and see
+movie titles they appeared in. You might want to index standard fields such as `actorName`, `genre` and `releaseYear`
+which will let you search by the actor's name or see all actors who appeared in movies during 2021. Similarly, you can
+index facet fields that will let you aggregate by “Genre” and “Year” so that you can show how many actors appeared in
+each year or genre. Few example documents:
+
+[source]
+----
+{ "name": "Tom Hanks", "genre": ["Comedy", "Drama", …], "year": [1988, 2000,…] }
+{ "name": "Harrison Ford", "genre": ["Action", "Adventure", …], "year": [1977, 1981, …] }
+----
+
+However, these facet fields do not allow you to show the following aggregation:
+
+.Number of Actors performing in movies by Genre and Year
+[cols="4*"]
+|===
+|           | 2020 | 2021 | 2022
+| Thriller  | 121  | 43   | 97
+| Action    | 145  | 52   | 130
+| Adventure | 87   | 21   | 32
+|===
+
+The reason is that each “genre” or “releaseYear” facet field is indexed in its own data structure, and therefore if an
+actor appeared in a "Thriller" movie in "2020" and "Action" movie in "2021", there's no way for you to tell that they
+didn't appear in an "Action" movie in "2020".
+
+[#automotive-parts]
+=== Automotive Parts Store
+
+Say you're building a search engine for an automotive parts store where customers can search for different car parts.
+For simplicity let's assume that each item in the catalog contains a searchable “type” field and “car model” it fits
+which consists of two separate fields: “manufacturer” and “year”. This lets you search for parts by their type as well
+as filter parts that fit only a certain manufacturer or year. Few example documents:
+
+[source]
+----
+{
+  "type": "Wiper Blades V1",
+  "models": [
+    { "manufaturer": "Ford", "year": 2010 },
+    { "manufacturer": "Chevy", "year": 2011 }
+  ]
+}
+{
+  "type": "Wiper Blades V2",
+  "models": [
+    { "manufaturer": "Ford", "year": 2011 },
+    { "manufacturer": "Chevy", "year": 2010 }
+  ]
+}
+----
+
+By breaking up the "models" field into its sub-fields "manufacturer" and "year", you can easily aggregate on parts that
+fit a certain manufacturer or year. However, if a user would like to aggregate on parts that can fit either a "Ford
+2010" or "Chevy 2011", then aggregating on the sub-fields will lead to a wrong count of 2 (in the above example) instead
+of 1.
+
+[#movie-awards]
+=== Movie Awards
+
+To showcase a 3-D multidimensional aggregation, lets expand the <<movie-actors>> example with awards an actor has
+received over the years. For this aggregation we will use four dimensions: Award Type ("Oscar", "Grammy", "Emmy"),
+Award Category ("Best Actor", "Best Supporting Actress"), Year and Genre. One interesting aggregation is to show how
+many "Best Actor" vs "Best Supporting Actor" awards one has received in the "Oscar" or "Emmy" for each year. Another
+aggregation is slicing the number of these awards by Genre over all the years.
+
+Building on these examples, one might be able to come up with an interesting use case for an N-dimensional aggregation
+(where `N > 3`). The higher `N` is, the harder it is to aggregate all the dimensions correctly and efficiently without
+`FacetSets`.
+
+== FacetSets API
+
+The `facetset` package consists of few components which allow you to index and aggregate multidimensional facet sets:
+
+=== FacetSet
+
+Holds a set of facet dimension values. Implementations are required to convert the dimensions into comparable long
+representation, as well can implement how the values are packed (encoded). The package offers four implementations:
+`Int/Float/Long/DoubleFacetSet` for `int`, `float`, `long` and `double` values respectively. You can also look at
+`org.apache.lucene.demo.facet.CustomFacetSetExample` in the `lucene/demo` package for a custom implementation of a
+`FacetSet`.
+
+=== FacetSetsField
+
+A `BinaryDocValues` field which lets you index a list of `FacetSet`. This field can be added to a document only once, so
+you will need to construct all the facet sets in advance.
+
+=== FacetSetMatcher
+
+Responsible for matching an encoded `FacetSet` against a given criteria. For example, `ExactFacetSetMatcher` only
+considers an encoded facet set as a match if all dimension values are equal to a given one. `RangeFacetSetMatcher`
+considers an encoded facet set as a match if all dimension values fall within predefined ranges. You can also look at
+`org.apache.lucene.demo.facet.CustomFacetSetExample` in the `lucene/demo` package for a custom implementation of a
+`FacetSetMatcher`.
+
+=== FacetSetDecoder

Review Comment:
   Good idea, done.



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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1155985365

   > but the one bigger question I'd like to discuss is how we envision handing different point types?
   
   I think there are two sides of supporting additional numeric types: indexing and aggregation. IMO it's still fine if `FSM` handles a `long[]`: indexing `doubles` will be done as `toSortableLong` and reading `int` and `float` into `long` is doable. Therefore on the aggregation side I feel like it's fine to keep the `long[]` matching API.
   
   For indexing we just need to convert the values to `byte[]`. We can do that by making `FacetSet` abstract with a `toBytes()` method and the current impl will be changed to `LongFacetSet`. To complement that on the aggregation side we will need to pass a _reader_ which can convert the `BytesRef` to a `long[]`. I'm thinking that the `Int/Float/Long/DoubleFacetSet` impls will do that.
   
   As for "mix-and-match" I think this provides a solution too, since the user will be able to implement their own `FacetSet` and convert their, as example, `int, long, long, float` facet set to `byte[]` and decode that back. I'll give it a try to see how it works.


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


[GitHub] [lucene] shaie merged pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie merged PR #841:
URL: https://github.com/apache/lucene/pull/841


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


[GitHub] [lucene] mdmarshmallow commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1156101105

   > To complement that on the aggregation side we will need to pass a reader which can convert the BytesRef to a long[]. I'm thinking that the Int/Float/Long/DoubleFacetSet impls will do that.
   
   Rather than do this, I think that it makes more sense for the impls to just have the ability of returning their own values as a `long[]` (something like `getComparableLongs()` that was in the original `hyperrectangle` package). That way, we won't need to run the reader on every `long[]` we read from the DV but instead can compare it directly with `FacetSet#getComparableLongs()`. I think this would be more efficient? 


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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r897537151


##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSetsField.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.facetset;
+
+import java.util.Arrays;
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A {@link BinaryDocValuesField} which encodes a list of {@link FacetSet facet sets}. The encoding
+ * scheme consists of a packed {@code long[]} where the first value denotes the number of dimensions
+ * in all the sets, followed by each set's values.
+ *
+ * @lucene.experimental
+ */
+public class FacetSetsField extends BinaryDocValuesField {
+
+  /**
+   * Create a new FacetSets field.
+   *
+   * @param name field name
+   * @param facetSets the {@link FacetSet} to index in that field. All must have the same number of
+   *     dimensions
+   * @throws IllegalArgumentException if the field name is null or the given facet sets are invalid
+   */
+  public static FacetSetsField create(String name, FacetSet... facetSets) {
+    validateFacetSets(facetSets);
+
+    return new FacetSetsField(name, toPackedLongs(facetSets));
+  }
+
+  private FacetSetsField(String name, BytesRef value) {
+    super(name, value);
+  }
+
+  private static void validateFacetSets(FacetSet... facetSets) {
+    if (facetSets == null || facetSets.length == 0) {
+      throw new IllegalArgumentException("FacetSets cannot be null or empty!");
+    }
+
+    int dims = facetSets[0].values.length;
+    if (!Arrays.stream(facetSets).allMatch(facetSet -> facetSet.values.length == dims)) {

Review Comment:
   Wasn't aware of this preference in the code base, will change to `anyMatch`



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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r898212017


##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/FloatFacetSet.java:
##########
@@ -0,0 +1,56 @@
+/*
+ * 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.facetset;
+
+import java.util.stream.IntStream;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.util.NumericUtils;
+
+/** A {@link FacetSet} which encodes float dimension values. */
+public class FloatFacetSet extends FacetSet {
+
+  /** The raw dimension values of this facet set. */
+  public final float[] floatValues;
+
+  /** Constructs a new instance of a facet set which stores {@code float} dimension values. */
+  public FloatFacetSet(float... values) {
+    super(toLongValues());
+
+    this.floatValues = values;
+  }
+
+  @Override
+  public int packValues(byte[] buf, int start) {
+    for (int i = 0, offset = start; i < floatValues.length; i++, offset += Float.BYTES) {
+      FloatPoint.encodeDimension(floatValues[i], buf, offset);
+    }
+    return floatValues.length * Float.BYTES;
+  }
+
+  @Override
+  public int sizePackedBytes() {
+    return values.length * Float.BYTES;
+  }
+
+  // Unfortunately there's no FloatStream to convert this in one line.
+  private static long[] toLongValues(float... values) {
+    if (values == null || values.length == 0) {
+      throw new IllegalArgumentException("values must not be null or empty");
+    }
+    return IntStream.range(0, values.length).mapToLong(NumericUtils::floatToSortableInt).toArray();

Review Comment:
   Shouldn't this be:
   
   ```suggestion
   return IntStream.range(0, values.length).mapToLong(idx -> NumericUtils.floatToSortableInt(values[idx])).toArray();
   ```



##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSetsField.java:
##########
@@ -0,0 +1,72 @@
+/*
+ * 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.facetset;
+
+import java.util.Arrays;
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A {@link BinaryDocValuesField} which encodes a list of {@link FacetSet facet sets}. The encoding
+ * scheme consists of a packed {@code byte[]} where the first value denotes the number of dimensions
+ * in all the sets, followed by each set's values.
+ *
+ * @lucene.experimental
+ */
+public class FacetSetsField extends BinaryDocValuesField {
+
+  /**
+   * Create a new FacetSets field.
+   *
+   * @param name field name
+   * @param facetSets the {@link FacetSet facet sets} to index in that field. All must have the same
+   *     number of dimensions
+   * @throws IllegalArgumentException if the field name is null or the given facet sets are invalid
+   */
+  public static FacetSetsField create(String name, FacetSet... facetSets) {
+    if (facetSets == null || facetSets.length == 0) {
+      throw new IllegalArgumentException("FacetSets cannot be null or empty!");
+    }
+
+    return new FacetSetsField(name, toPackedValues(facetSets));
+  }
+
+  private FacetSetsField(String name, BytesRef value) {
+    super(name, value);
+  }
+
+  private static BytesRef toPackedValues(FacetSet... facetSets) {
+    int numDims = facetSets[0].dims;
+    // We could have created a buffer that can accommodate Long.BYTES per dimension value in each
+    // facet set. The below attempts to avoid allocating unnecessarily bigger arrays.
+    byte[] buf = new byte[4 + Arrays.stream(facetSets).mapToInt(FacetSet::sizePackedBytes).sum()];

Review Comment:
   Maybe use `Integer.BYTES` instead of `4` here?



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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1142451787

   I pushed a new package with the `FacetSet` API I had in mind. As I wrote before, while thinking about it I realized that there are few issues to handle, so I've decided to implement one of the examples in the document: the Automotive Parts Store, just to get a feel how it would work. Few points:
   
   1. The `MatchingFacetSetsCounts` takes a `FacetSetMatcher` and already support multi-valued fields. For now I've implemented an `ExactFacetSetMathcer` which requires all dim values to match a given set.
     * We can implement additional ones, such as `RangeFacetSetMatcher` which evaluates each value against a range.
     * We cal implement the Movie Awards matcher which looks at some of the dimension values to determine a match.
   
   2. The `MatchingFacetSetsCounts` is not a "one-Counts-to-rule-them-all" class. It's just one use case which counts how many documents each `matcher` matched as well as how many documents matched overall.
     * We can, and should!, also implement a `TopMatchingFacetSetCounts` which fixes some dimensions of a set and computes the counts for the "free" dimensions. E.g. "Top 3 years for actors performing in Thriller movies": the matcher will evaluate as a match the first dimensions (the "Genre") and compute a `counts[]` for the "Year" dimension, then its `getTopChildren` will return the 3 years with the most "Thriller" actors.
   
   Eventually I see more "XYZFacetCounts" implementation, and not necessarily many more `FacetSetMatcher` impls.
   
   NOTE: all names in the commit are just proposals, feel free to propose better ones.


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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1146061771

   Hi Greg, thanks for your comments. Earlier today I tried to play with the
   new API to implement some other use cases just to get a feel for how they
   will work, and I realized why `HyperRectangle` was proposed and implemented
   the way it was (sorry for being too slow!). Let me try to clarify my
   thoughts a bit, from multiple perspectives:
   
   * HyperRectangles are indeed a generic way of matching an N-dimensional
   point. If one wants ranges, one passes a pair where min/max are different.
   If one wants an exact match, one would pass a range where min/max are equal.
     * What I proposed with the `ExactFacetSetMatcher` implementation is
   merely a specialization of the above. So instead of passing ranges where
   min/max are the same, and having the aggregation algo do redundant range
   checks, it just specializes on how the aggregation is done. Additionally,
   from an API perspective, it might be clearer to the user that they only
   need to pass the expected values, and not construct ranges "because that's
   what the API allows".
     * We could have let `LongPair` implement a `match()` API itself, and a
   sugar API for `LongPair.create(min, max)` which will return either a
   `RangeLongPair` or `ExactLongPair` (don't mind the names too much) to
   specialize the impl, but I'm not sure what will perform better -- calling
   the `Pair.match()` or just doing range checks always.
   
   * From an API perspective and the user, I wonder if HyperRectangle is a
   clear enough name to denote what we're building here. I.e., is it perhaps
   too expert? For instance I initially thought the proposal is for
   geo-something faceting before I realized it has nothing specifically to do
   with geo (again, sorry for being slow :)). Naming is hard, but I _think_
   that `FacetSet` with a bunch of helper classes might make the API clearer.
      * I totally think we should have a `HyperRectangle` impl, maybe call it
   `HyperRectangleFacetSetMatcher` or `RangeFacetSetMatcher`. This is the
   generic catch-all / fallback impl if one cannot find a specialized impl, or
   doesn't know how to write one.
     * I hope that with this API we'll also pave the way for users to realize
   they can implement their own `FacetSetMatcher`, for instance treating the
   first 2 dimensions as range, and 3rd and 4th as exact (again, to create
   specialized matchers).
     * I also think that the proposed API under `facetset` is easier to
   extend, even though I'm sure we can re-structure the `hyperrectangle`
   package to allow for such extension. Essentially you want a _Reader_ which
   knows how to read the `long[]` and a `Filter/Matcher/Whatever` which
   interprets them, returning a boolean or something else. That part is the
   extension point we'd hope users to implement, which will make the
   underlying storage of the points an abstraction that users don't have to
   deal with.
   
   * Regarding the other use cases I've mentioned, both `HyperRectangle` and
   `MatchingFacetSetCounts` do the same job -- they match an entire set of
   points against a given set of points. The `Matcher` even implements an API
   which returns a `boolean`. True, you can pass for some of the dimensions
   `(NEG_INF, POS_INF)` to denote that you "don't care" about some of the
   dimensions, but still at its core this implementation tells you how many
   docs matched each set.
     * What this impl doesn't let you do is use, say dims 1-3 for matching,
   and 4 for counting so that you can ask "What are the top-3 Years for
   Oscar+Drama awards" (I hope what I wrote makes sense!!). In this example
   you'll want to "match" docs if they have "Oscar" and "Drama" dimensions,
   but then count the "Year" dimension and compute the top-K. This use case
   cannot be implemented with neither of the current proposed impls, since
   they only match docs.
     * What I tried to say is that for this kind of use case we'll need a diff
   counting impl (but still use the same on-disk structure!), that's all. One
   that keeps track of the "Year" counts and its `getTopChildren` returns the
   top 3 Years. I hope that makes sense?
   
   I'll add the HyperRectangle impl to the `facetset` package (I'll reuse the
   existing classes from `hyperrectangle` for now and we can see how it works?
   
   On Fri, Jun 3, 2022 at 10:56 AM Greg Miller ***@***.***>
   wrote:
   
   > Trying to catch up on this now. I've been traveling and it's been
   > difficult to find time. Thanks for all your thoughts @shaie
   > <https://github.com/shaie>!
   >
   > I think I'm only half-following your thoughts on the different APIs
   > necessary, and will probably need to look at what you've documented in more
   > detail. But... as a half-baked response, I'm not convinced (yet?) that we
   > need this level of complexity in the API. In my mind, what we're trying to
   > build is a generalization of what is already supported in long/double-range
   > faceting (e.g., LongRangeFacetCounts), where the user specifies all the
   > ranges they want counts for, we count hits against those ranges, and
   > support returning those counts through a couple APIs. Those faceting
   > implementations allow ranges to be specified in a single dimension, and
   > determine which ranges the document points (in one-dimensional space) fall
   > in.
   >
   > So "hyperrectangle faceting"—in my original thinking at least—is just a
   > generalization of this to multiple dimensions. The points associated with
   > the documents are in n-dimensional space, and the user specifies the
   > different "hyperrectangles" they want counts for by providing a [min, max]
   > range in each dimension. For cases like the "automotive parts finder"
   > example, it's perfectly valid for the "hyperrectangles" provided by the
   > user to also be single points (where the min/max are equivalent values in
   > each dimension). But it's also valid to mix-and-match, where some
   > dimensions are single points and some are ranges (e.g., "all auto parts
   > that fit 'Chevy' (single point) for the years 2000 - 2010 (range)).
   >
   > In the situation where a user wants to "fix some dimension" and count over
   > others, it can still be described as a set of "hyperrectangles," but where
   > the specified ranges on some of the dimensions happen to be the same across
   > all of them.
   >
   > So I'm not quite sure if what you're suggesting in the API is just
   > syntactic sugar on top of this idea, or if we're possibly talking about
   > different things here? I'll try to dive into your suggestion more though
   > and understand. I feel like I'm just missing something important and need
   > to catch up on your thinking. Thanks again for sharing! I'll circle back in
   > a few days when I've (hopefully) had some more time to spend on this :)
   >
   > —
   > Reply to this email directly, view it on GitHub
   > <https://github.com/apache/lucene/pull/841#issuecomment-1145696479>, or
   > unsubscribe
   > <https://github.com/notifications/unsubscribe-auth/AA2PE3EZ4453PDPJ6ETLSP3VNG3BPANCNFSM5UNJB2OA>
   > .
   > You are receiving this because you were mentioned.Message ID:
   > ***@***.***>
   >
   


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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1151548324

   Regarding this "long matches API", I'm fairly confused. Why would we need this? I think I'm missing a chunk of previous conversation? But looking at this latest code, I can't imagine why we'd need this?
   
   Is this about providing enough information to optimize with KD/R-Trees? If that's the case, I don't think we want a `matches(long[])` method right? We just need a way for the `FSM` instances to expose their bounding-boxes. I don't think we even need to store the original long[] array to do this. But I don't think we need to hash that out here.
   
   So unless I'm totally missing something, I think we should keep the `matches(byte[])` method.


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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r873326117


##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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 org.apache.lucene.util.NumericUtils;
+
+/** Stores a hyper rectangle as an array of DoubleRangePairs */
+public class DoubleHyperRectangle extends HyperRectangle {
+
+  /** Stores pair as LongRangePair */
+  private final DoubleRangePair[] pairs;
+
+  /** Created DoubleHyperRectangle */
+  public DoubleHyperRectangle(String label, DoubleRangePair... pairs) {
+    super(label, pairs.length);
+    this.pairs = pairs;
+  }
+
+  @Override
+  public LongHyperRectangle.LongRangePair getComparableDimRange(int dim) {
+    long longMin = NumericUtils.doubleToSortableLong(pairs[dim].min);
+    long longMax = NumericUtils.doubleToSortableLong(pairs[dim].max);
+    return new LongHyperRectangle.LongRangePair(longMin, true, longMax, true);
+  }
+
+  /** Defines a single range in a DoubleHyperRectangle */
+  public static class DoubleRangePair {
+    /** Inclusive min */
+    public final double min;
+
+    /** Inclusive max */
+    public final double max;
+
+    /**
+     * Creates a LongRangePair, very similar to the constructor of {@link
+     * org.apache.lucene.facet.range.DoubleRange}
+     *
+     * @param minIn Min value of pair
+     * @param minInclusive If minIn is inclusive
+     * @param maxIn Max value of pair
+     * @param maxInclusive If maxIn is inclusive
+     */
+    public DoubleRangePair(double minIn, boolean minInclusive, double maxIn, boolean maxInclusive) {
+      if (Double.isNaN(minIn)) {
+        throw new IllegalArgumentException("min cannot be NaN");
+      }
+      if (!minInclusive) {
+        minIn = Math.nextUp(minIn);
+      }
+
+      if (Double.isNaN(maxIn)) {

Review Comment:
   Move this check before `is (!minInclusive)` and I suggest unifying both checks like this:
   
   ```
   if (Double.isNaN(minIn) || Double.isNaN(maxIn)) {
     throw new IllegalArgumentException("min and max cannot be NaN: min=" + minIn + ", max=" + maxIn);
   }
   ```



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.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 by subclass. */

Review Comment:
   Either remove "in" or "by", I think one of them is redundant



##########
lucene/facet/src/test/org/apache/lucene/facet/hyperrectangle/TestHyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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 org.apache.lucene.document.Document;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.FacetTestCase;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollectorManager;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.tests.index.RandomIndexWriter;
+
+public class TestHyperRectangleFacetCounts extends FacetTestCase {
+
+  public void testBasicLong() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+
+    for (long l = 0; l < 100; l++) {
+      Document doc = new Document();
+      LongPointFacetField field = new LongPointFacetField("field", l, l + 1L, l + 2L);
+      doc.add(field);
+      w.addDocument(doc);
+    }
+
+    // Also add point with Long.MAX_VALUE
+    Document doc = new Document();
+    LongPointFacetField field =
+        new LongPointFacetField("field", Long.MAX_VALUE - 2L, Long.MAX_VALUE - 1L, Long.MAX_VALUE);
+    doc.add(field);
+    w.addDocument(doc);
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    FacetsCollector fc = s.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    Facets facets =
+        new HyperRectangleFacetCounts(
+            "field",
+            fc,
+            new LongHyperRectangle(
+                "less than (10, 11, 12)",
+                new LongHyperRectangle.LongRangePair(0L, true, 10L, false),
+                new LongHyperRectangle.LongRangePair(0L, true, 11L, false),
+                new LongHyperRectangle.LongRangePair(0L, true, 12L, false)),
+            new LongHyperRectangle(
+                "less than or equal to (10, 11, 12)",
+                new LongHyperRectangle.LongRangePair(0L, true, 10L, true),
+                new LongHyperRectangle.LongRangePair(0L, true, 11L, true),
+                new LongHyperRectangle.LongRangePair(0L, true, 12L, true)),
+            new LongHyperRectangle(
+                "over (90, 91, 92)",
+                new LongHyperRectangle.LongRangePair(90L, false, 100L, false),
+                new LongHyperRectangle.LongRangePair(91L, false, 101L, false),
+                new LongHyperRectangle.LongRangePair(92L, false, 102L, false)),
+            new LongHyperRectangle(
+                "(90, 91, 92) or above",
+                new LongHyperRectangle.LongRangePair(90L, true, 100L, false),
+                new LongHyperRectangle.LongRangePair(91L, true, 101L, false),
+                new LongHyperRectangle.LongRangePair(92L, true, 102L, false)),
+            new LongHyperRectangle(
+                "over (1000, 1000, 1000)",
+                new LongHyperRectangle.LongRangePair(1000L, false, Long.MAX_VALUE - 2L, true),
+                new LongHyperRectangle.LongRangePair(1000L, false, Long.MAX_VALUE - 1L, true),
+                new LongHyperRectangle.LongRangePair(1000L, false, Long.MAX_VALUE, true)));
+
+    FacetResult result = facets.getTopChildren(10, "field");
+    assertEquals(
+        """
+                        dim=field path=[] value=22 childCount=5
+                          less than (10, 11, 12) (10)
+                          less than or equal to (10, 11, 12) (11)
+                          over (90, 91, 92) (9)
+                          (90, 91, 92) or above (10)
+                          over (1000, 1000, 1000) (1)
+                        """,
+        result.toString());
+
+    // test getTopChildren(0, dim)
+    expectThrows(
+        IllegalArgumentException.class,
+        () -> {
+          facets.getTopChildren(0, "field");
+        });
+
+    r.close();
+    d.close();
+  }
+
+  public void testBasicDouble() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+
+    for (double l = 0; l < 100; l++) {
+      Document doc = new Document();
+      DoublePointFacetField field = new DoublePointFacetField("field", l, l + 1.0, l + 2.0);
+      doc.add(field);
+      w.addDocument(doc);
+    }
+
+    // Also add point with Long.MAX_VALUE
+    Document doc = new Document();
+    DoublePointFacetField field =
+        new DoublePointFacetField(
+            "field", Double.MAX_VALUE - 2.0, Double.MAX_VALUE - 1.0, Double.MAX_VALUE);
+    doc.add(field);
+    w.addDocument(doc);
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    FacetsCollector fc = s.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    Facets facets =
+        new HyperRectangleFacetCounts(
+            "field",
+            fc,
+            new DoubleHyperRectangle(
+                "less than (10, 11, 12)",
+                new DoubleHyperRectangle.DoubleRangePair(0.0, true, 10.0, false),
+                new DoubleHyperRectangle.DoubleRangePair(0.0, true, 11.0, false),
+                new DoubleHyperRectangle.DoubleRangePair(0.0, true, 12.0, false)),
+            new DoubleHyperRectangle(
+                "less than or equal to (10, 11, 12)",
+                new DoubleHyperRectangle.DoubleRangePair(0.0, true, 10.0, true),
+                new DoubleHyperRectangle.DoubleRangePair(0.0, true, 11.0, true),
+                new DoubleHyperRectangle.DoubleRangePair(0.0, true, 12.0, true)),
+            new DoubleHyperRectangle(
+                "over (90, 91, 92)",
+                new DoubleHyperRectangle.DoubleRangePair(90.0, false, 100.0, false),
+                new DoubleHyperRectangle.DoubleRangePair(91.0, false, 101.0, false),
+                new DoubleHyperRectangle.DoubleRangePair(92.0, false, 102.0, false)),
+            new DoubleHyperRectangle(
+                "(90, 91, 92) or above",
+                new DoubleHyperRectangle.DoubleRangePair(90.0, true, 100.0, false),
+                new DoubleHyperRectangle.DoubleRangePair(91.0, true, 101.0, false),
+                new DoubleHyperRectangle.DoubleRangePair(92.0, true, 102.0, false)),
+            new DoubleHyperRectangle(
+                "over (1000, 1000, 1000)",
+                new DoubleHyperRectangle.DoubleRangePair(
+                    1000.0, false, Double.MAX_VALUE - 2.0, true),
+                new DoubleHyperRectangle.DoubleRangePair(
+                    1000.0, false, Double.MAX_VALUE - 1.0, true),
+                new DoubleHyperRectangle.DoubleRangePair(1000.0, false, Double.MAX_VALUE, true)));
+
+    FacetResult result = facets.getTopChildren(10, "field");
+    assertEquals(
+        """
+                        dim=field path=[] value=22 childCount=5

Review Comment:
   nit: personally I prefer less this type of assertions as they are very fragile. If we change the `toString()` tomorrow we'll need to fix all the tests. Can we change to test to make explicit assertions on the label + count? Eventually we want to test the returned facets, not their `toString()` representation.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangle.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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;
+
+/** Holds the name and the number of dims for a HyperRectangle */
+public abstract class HyperRectangle {
+  /** Label that identifies this range. */
+  public final String label;
+
+  /** How many dimensions this hyper rectangle has (IE: a regular rectangle would have dims=2) */
+  public final int dims;
+
+  /** Sole constructor. */
+  protected HyperRectangle(String label, int dims) {
+    if (label == null) {
+      throw new NullPointerException("label must not be null");
+    }
+    this.label = label;
+    this.dims = dims;

Review Comment:
   Do we want to add some validity checks on dims?



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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 org.apache.lucene.util.NumericUtils;
+
+/** Stores a hyper rectangle as an array of DoubleRangePairs */
+public class DoubleHyperRectangle extends HyperRectangle {
+
+  /** Stores pair as LongRangePair */

Review Comment:
   DoubleRangePair? 



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangle.java:
##########
@@ -0,0 +1,43 @@
+/*
+ * 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;
+
+/** Holds the name and the number of dims for a HyperRectangle */
+public abstract class HyperRectangle {
+  /** Label that identifies this range. */
+  public final String label;
+
+  /** How many dimensions this hyper rectangle has (IE: a regular rectangle would have dims=2) */
+  public final int dims;
+
+  /** Sole constructor. */
+  protected HyperRectangle(String label, int dims) {
+    if (label == null) {
+      throw new NullPointerException("label must not be null");

Review Comment:
   I find it consistent that you throw NPE here and IAE in `DoubleRangePair` for illegal arguments.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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 org.apache.lucene.util.NumericUtils;
+
+/** Stores a hyper rectangle as an array of DoubleRangePairs */
+public class DoubleHyperRectangle extends HyperRectangle {
+
+  /** Stores pair as LongRangePair */
+  private final DoubleRangePair[] pairs;
+
+  /** Created DoubleHyperRectangle */
+  public DoubleHyperRectangle(String label, DoubleRangePair... pairs) {
+    super(label, pairs.length);

Review Comment:
   Q: can `pairs` be null? If so perhaps we should add a null check? Also, is it a valid case to receive an empty `pairs`?



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.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 by 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)

Review Comment:
   I think `hits` should be named `facetsCollector`? Since these are not the actual hits to collect facets on. If you accept this, please rename in all the places



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.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 by 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    this.field = field;
+    this.hyperRectangles = hyperRectangles;
+    this.dims = hyperRectangles[0].dims;
+    for (HyperRectangle hyperRectangle : hyperRectangles) {

Review Comment:
   nit: currently you loop through this list always, even when `-ea` is not set. If you factor it out to a method you could call `assert hyperRectanglesDims(this.dims)`



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.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 by 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)
+      throws IOException {
+    assert hyperRectangles.length > 0 : "Hyper rectangle ranges cannot be empty";
+    this.field = field;
+    this.hyperRectangles = hyperRectangles;
+    this.dims = hyperRectangles[0].dims;
+    for (HyperRectangle hyperRectangle : hyperRectangles) {
+      assert hyperRectangle.dims == this.dims
+          : "All hyper rectangles must be the same dimensionality";
+    }
+    this.counts = new int[hyperRectangles.length];
+    count(field, hits.getMatchingDocs());
+  }
+
+  /** 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; ) {
+        if (binaryDocValues.advanceExact(doc)) {
+          long[] point = LongPoint.unpack(binaryDocValues.binaryValue());
+          assert point.length == dims : "Point dimension is incompatible with hyper rectangle";

Review Comment:
   nit: is it useful to add the `point.length` and `dims` to the assertion message?



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,163 @@
+/*
+ * 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.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 by 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
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of long hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, LongHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  /**
+   * Create HyperRectangleFacetCounts using
+   *
+   * @param field Field name
+   * @param hits Hits to facet on
+   * @param hyperRectangles List of double hyper rectangle facets
+   * @throws IOException If there is a problem reading the field
+   */
+  public HyperRectangleFacetCounts(
+      String field, FacetsCollector hits, DoubleHyperRectangle... hyperRectangles)
+      throws IOException {
+    this(true, field, hits, hyperRectangles);
+  }
+
+  private HyperRectangleFacetCounts(
+      boolean discarded, String field, FacetsCollector hits, HyperRectangle... hyperRectangles)

Review Comment:
   I see `discarded` is always `true` now, am I missing a call?



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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 org.apache.lucene.util.NumericUtils;
+
+/** Stores a hyper rectangle as an array of DoubleRangePairs */
+public class DoubleHyperRectangle extends HyperRectangle {
+
+  /** Stores pair as LongRangePair */
+  private final DoubleRangePair[] pairs;
+
+  /** Created DoubleHyperRectangle */
+  public DoubleHyperRectangle(String label, DoubleRangePair... pairs) {
+    super(label, pairs.length);
+    this.pairs = pairs;
+  }
+
+  @Override
+  public LongHyperRectangle.LongRangePair getComparableDimRange(int dim) {

Review Comment:
   perf: this method is called many times, IIUC once for each dimension for each requested rectangle for each matching document. Consider passing a `LongPair` instance in, and only update its fields.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/LongHyperRectangle.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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;
+
+/** Stores a hyper rectangle as an array of LongRangePairs */
+public class LongHyperRectangle extends HyperRectangle {
+
+  private final LongRangePair[] pairs;
+
+  /** Created LongHyperRectangle */
+  public LongHyperRectangle(String label, LongRangePair... pairs) {
+    super(label, pairs.length);
+    this.pairs = pairs;
+  }
+
+  @Override
+  public LongRangePair getComparableDimRange(int dim) {
+    return pairs[dim];
+  }
+
+  /** Defines a single range in a LongHyperRectangle */
+  public static class LongRangePair {
+    /** Inclusive min */
+    public final long min;
+
+    /** Inclusive max */
+    public final long max;
+
+    /**
+     * Creates a LongRangePair, very similar to the constructor of {@link
+     * org.apache.lucene.facet.range.LongRange}
+     *
+     * @param minIn Min value of pair
+     * @param minInclusive If minIn is inclusive
+     * @param maxIn Max value of pair
+     * @param maxInclusive If maxIn is inclusive
+     */
+    public LongRangePair(long minIn, boolean minInclusive, long maxIn, boolean maxInclusive) {
+      if (!minInclusive) {
+        if (minIn != Long.MAX_VALUE) {
+          minIn++;
+        } else {
+          throw new IllegalArgumentException("Invalid min input");
+        }
+      }
+
+      if (!maxInclusive) {
+        if (maxIn != Long.MIN_VALUE) {
+          maxIn--;
+        } else {
+          throw new IllegalArgumentException("Invalid max input");

Review Comment:
   Add the actual input value to the exception for clarity



##########
lucene/facet/src/test/org/apache/lucene/facet/hyperrectangle/TestHyperRectangleFacetCounts.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * 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 org.apache.lucene.document.Document;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.FacetTestCase;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollectorManager;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.tests.index.RandomIndexWriter;
+
+public class TestHyperRectangleFacetCounts extends FacetTestCase {
+
+  public void testBasicLong() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+
+    for (long l = 0; l < 100; l++) {
+      Document doc = new Document();
+      LongPointFacetField field = new LongPointFacetField("field", l, l + 1L, l + 2L);
+      doc.add(field);
+      w.addDocument(doc);
+    }
+
+    // Also add point with Long.MAX_VALUE
+    Document doc = new Document();
+    LongPointFacetField field =
+        new LongPointFacetField("field", Long.MAX_VALUE - 2L, Long.MAX_VALUE - 1L, Long.MAX_VALUE);
+    doc.add(field);
+    w.addDocument(doc);
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    FacetsCollector fc = s.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    Facets facets =
+        new HyperRectangleFacetCounts(
+            "field",
+            fc,
+            new LongHyperRectangle(
+                "less than (10, 11, 12)",
+                new LongHyperRectangle.LongRangePair(0L, true, 10L, false),
+                new LongHyperRectangle.LongRangePair(0L, true, 11L, false),
+                new LongHyperRectangle.LongRangePair(0L, true, 12L, false)),
+            new LongHyperRectangle(
+                "less than or equal to (10, 11, 12)",
+                new LongHyperRectangle.LongRangePair(0L, true, 10L, true),
+                new LongHyperRectangle.LongRangePair(0L, true, 11L, true),
+                new LongHyperRectangle.LongRangePair(0L, true, 12L, true)),
+            new LongHyperRectangle(
+                "over (90, 91, 92)",
+                new LongHyperRectangle.LongRangePair(90L, false, 100L, false),
+                new LongHyperRectangle.LongRangePair(91L, false, 101L, false),
+                new LongHyperRectangle.LongRangePair(92L, false, 102L, false)),
+            new LongHyperRectangle(
+                "(90, 91, 92) or above",
+                new LongHyperRectangle.LongRangePair(90L, true, 100L, false),
+                new LongHyperRectangle.LongRangePair(91L, true, 101L, false),
+                new LongHyperRectangle.LongRangePair(92L, true, 102L, false)),
+            new LongHyperRectangle(
+                "over (1000, 1000, 1000)",
+                new LongHyperRectangle.LongRangePair(1000L, false, Long.MAX_VALUE - 2L, true),
+                new LongHyperRectangle.LongRangePair(1000L, false, Long.MAX_VALUE - 1L, true),
+                new LongHyperRectangle.LongRangePair(1000L, false, Long.MAX_VALUE, true)));
+
+    FacetResult result = facets.getTopChildren(10, "field");
+    assertEquals(
+        """
+                        dim=field path=[] value=22 childCount=5
+                          less than (10, 11, 12) (10)
+                          less than or equal to (10, 11, 12) (11)
+                          over (90, 91, 92) (9)
+                          (90, 91, 92) or above (10)
+                          over (1000, 1000, 1000) (1)
+                        """,
+        result.toString());
+
+    // test getTopChildren(0, dim)
+    expectThrows(
+        IllegalArgumentException.class,
+        () -> {
+          facets.getTopChildren(0, "field");
+        });
+
+    r.close();
+    d.close();
+  }
+
+  public void testBasicDouble() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+
+    for (double l = 0; l < 100; l++) {
+      Document doc = new Document();
+      DoublePointFacetField field = new DoublePointFacetField("field", l, l + 1.0, l + 2.0);
+      doc.add(field);
+      w.addDocument(doc);
+    }
+
+    // Also add point with Long.MAX_VALUE
+    Document doc = new Document();
+    DoublePointFacetField field =
+        new DoublePointFacetField(
+            "field", Double.MAX_VALUE - 2.0, Double.MAX_VALUE - 1.0, Double.MAX_VALUE);
+    doc.add(field);
+    w.addDocument(doc);
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    FacetsCollector fc = s.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    Facets facets =
+        new HyperRectangleFacetCounts(
+            "field",
+            fc,
+            new DoubleHyperRectangle(
+                "less than (10, 11, 12)",
+                new DoubleHyperRectangle.DoubleRangePair(0.0, true, 10.0, false),
+                new DoubleHyperRectangle.DoubleRangePair(0.0, true, 11.0, false),
+                new DoubleHyperRectangle.DoubleRangePair(0.0, true, 12.0, false)),
+            new DoubleHyperRectangle(
+                "less than or equal to (10, 11, 12)",
+                new DoubleHyperRectangle.DoubleRangePair(0.0, true, 10.0, true),
+                new DoubleHyperRectangle.DoubleRangePair(0.0, true, 11.0, true),
+                new DoubleHyperRectangle.DoubleRangePair(0.0, true, 12.0, true)),
+            new DoubleHyperRectangle(
+                "over (90, 91, 92)",
+                new DoubleHyperRectangle.DoubleRangePair(90.0, false, 100.0, false),
+                new DoubleHyperRectangle.DoubleRangePair(91.0, false, 101.0, false),
+                new DoubleHyperRectangle.DoubleRangePair(92.0, false, 102.0, false)),
+            new DoubleHyperRectangle(
+                "(90, 91, 92) or above",
+                new DoubleHyperRectangle.DoubleRangePair(90.0, true, 100.0, false),
+                new DoubleHyperRectangle.DoubleRangePair(91.0, true, 101.0, false),
+                new DoubleHyperRectangle.DoubleRangePair(92.0, true, 102.0, false)),
+            new DoubleHyperRectangle(
+                "over (1000, 1000, 1000)",
+                new DoubleHyperRectangle.DoubleRangePair(
+                    1000.0, false, Double.MAX_VALUE - 2.0, true),
+                new DoubleHyperRectangle.DoubleRangePair(
+                    1000.0, false, Double.MAX_VALUE - 1.0, true),
+                new DoubleHyperRectangle.DoubleRangePair(1000.0, false, Double.MAX_VALUE, true)));
+
+    FacetResult result = facets.getTopChildren(10, "field");
+    assertEquals(
+        """
+                        dim=field path=[] value=22 childCount=5
+                          less than (10, 11, 12) (10)
+                          less than or equal to (10, 11, 12) (11)
+                          over (90, 91, 92) (9)
+                          (90, 91, 92) or above (10)
+                          over (1000, 1000, 1000) (1)
+                        """,
+        result.toString());
+
+    // test getTopChildren(0, dim)
+    expectThrows(

Review Comment:
   I think this test is just repeated for the Long + Double cases. If you factor it out to its own test I think (1) it will be clearer what is being tested, e.g. `testInvalidTopN` and (2) you won't need to test it twice.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/LongHyperRectangle.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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;
+
+/** Stores a hyper rectangle as an array of LongRangePairs */
+public class LongHyperRectangle extends HyperRectangle {
+
+  private final LongRangePair[] pairs;
+
+  /** Created LongHyperRectangle */
+  public LongHyperRectangle(String label, LongRangePair... pairs) {
+    super(label, pairs.length);
+    this.pairs = pairs;
+  }
+
+  @Override
+  public LongRangePair getComparableDimRange(int dim) {
+    return pairs[dim];
+  }
+
+  /** Defines a single range in a LongHyperRectangle */
+  public static class LongRangePair {
+    /** Inclusive min */
+    public final long min;
+
+    /** Inclusive max */
+    public final long max;
+
+    /**
+     * Creates a LongRangePair, very similar to the constructor of {@link
+     * org.apache.lucene.facet.range.LongRange}
+     *
+     * @param minIn Min value of pair
+     * @param minInclusive If minIn is inclusive
+     * @param maxIn Max value of pair
+     * @param maxInclusive If maxIn is inclusive
+     */
+    public LongRangePair(long minIn, boolean minInclusive, long maxIn, boolean maxInclusive) {
+      if (!minInclusive) {
+        if (minIn != Long.MAX_VALUE) {
+          minIn++;
+        } else {
+          throw new IllegalArgumentException("Invalid min input");

Review Comment:
   Add the actual input value to the exception for clarity



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


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

Posted by GitBox <gi...@apache.org>.
gsmiller commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1154143249

   Ah, sorry... I accidentally hit the "close" button! My bad. Reopened.


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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r897535013


##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/MatchingFacetSetsCounts.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.facetset;
+
+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.ConjunctionUtils;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Returns the counts for each given {@link FacetSet}
+ *
+ * @lucene.experimental
+ */
+public class MatchingFacetSetsCounts extends Facets {
+
+  private final FacetSetMatcher[] facetSetMatchers;
+  private final int[] counts;
+  private final String field;
+  private final int totCount;
+
+  /**
+   * Constructs a new instance of matching facet set counts which calculates the countBytes for each
+   * given facet set matcher.
+   */
+  public MatchingFacetSetsCounts(
+      String field, FacetsCollector hits, FacetSetMatcher... facetSetMatchers) throws IOException {
+    if (facetSetMatchers == null || facetSetMatchers.length == 0) {
+      throw new IllegalArgumentException("facetSetMatchers cannot be null or empty");
+    }
+    if (areFacetSetMatcherDimensionsInconsistent(facetSetMatchers)) {
+      throw new IllegalArgumentException("All facet set matchers must be the same dimensionality");
+    }
+    this.field = field;
+    this.facetSetMatchers = facetSetMatchers;
+    this.counts = new int[facetSetMatchers.length];
+    this.totCount = count(field, hits.getMatchingDocs());
+  }
+
+  /** Counts from the provided field. */
+  private int count(String field, List<FacetsCollector.MatchingDocs> matchingDocs)

Review Comment:
   I see your point. I did that mainly to keep fields `final` to denote that are not changing after initialization. I realize there's a "side effect" of populating the counts array in the method which sucks (cause we can't return two values from a method). Is it better though over having all fields `final`?



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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r897535157


##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/MatchingFacetSetsCounts.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.facetset;
+
+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.ConjunctionUtils;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Returns the counts for each given {@link FacetSet}
+ *
+ * @lucene.experimental
+ */
+public class MatchingFacetSetsCounts extends Facets {
+
+  private final FacetSetMatcher[] facetSetMatchers;
+  private final int[] counts;
+  private final String field;
+  private final int totCount;
+
+  /**
+   * Constructs a new instance of matching facet set counts which calculates the countBytes for each

Review Comment:
   This is an IDEA refactor side-effect, obviously an error :).



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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r897535559


##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/RangeFacetSetMatcher.java:
##########
@@ -0,0 +1,112 @@
+/*
+ * 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.facetset;
+
+import java.util.Arrays;
+
+/**
+ * A {@link FacetSetMatcher} which considers a set as a match if all dimensions fall within the
+ * given corresponding range.
+ *
+ * @lucene.experimental
+ */
+public class RangeFacetSetMatcher extends FacetSetMatcher {
+
+  private final long[] lowerRanges;
+  private final long[] upperRanges;
+
+  /**
+   * Constructs and instance to match facet sets with dimensions that fall within the given ranges.

Review Comment:
   Done



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


[GitHub] [lucene] mdmarshmallow commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1150471678

   So based on everyone's comments:
   
   1. It seems like we should ditch the `hyperrectangle` implementation and that `facetset` does everything we need for right now
   2. When we decide to optimize this (right after this PR is merged ideally), we would let `MatchingFacetSetsCount` be able to take a look at the `FSM`'s passed to it and then determine if it should put the FSM into an R tree, KD tree, or just linearly scan based on the `min` and `max` of each`FSM`. I think this makes sense, but we also shouldn't discuss it too much here as I think this is for another PR. I think the point is we can optimize the `facetsets` package in it's current state. With that being said, I do plan on writing the KD and R tree optimizations as soon as this is merged so I am still for this remaining a `long[]` API.


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


[GitHub] [lucene] mdmarshmallow commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1149153824

   I want to summarize the open questions we have right now to help figure out what we should do next:
   1. Should we split the `ExactFSM` and range multidim implementations into separate packages (`facetset` for the exact implementation and `hyperrectangle` for the range implementation) or the same?
   2. As @shaie mentioned, should we have a `long[]` based API or not?
   
   For the first one, I talked with Greg a bit more about his suggestion to have them in separate packages and I think I agree with this. We can then make more specialized subclasses (like `DoubleExactFSM` and `DoubleHyperRectangle`) without cluttering up the package, and optimizations won't have to account for the fact we could be doing exact matching vs range matching etc. Maybe we combine them in the future? But I think for now we should keep them separate.
   
   For the second question, I think we should keep this as a `long[]` based API as we know we want to make the KD tree and R tree optimizations in the future, so adding extra work for us to have to revert doesn't make sense to me? Though if you guys have contrary opinions please let me know, I could see other viewpoints for this.
   
   I think once we come to agreement on these questions it will be a lot easier to move forward, at least for me cause I think it will help me have a greater understanding of exactly what our final product (for this PR at least) should look like.


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


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

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r878767917


##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangle.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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;
+
+/** Holds the name and the number of dims for a HyperRectangle */

Review Comment:
   Made this comment more accurate



##########
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";

Review Comment:
   I think that these should just throw `IllegalArgumentExceptions`, I changed this to a conditional and included a `null` check.



##########
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:
   For right now yes. I was planning on adding multi value support after the basic API got fleshed out (maybe in a separate issue?)



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoublePointFacetField.java:
##########
@@ -0,0 +1,44 @@
+/*
+ * 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.util.Arrays;
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * Takes an array of doubles and converts them to sortable longs, then stores as a {@link
+ * BinaryDocValuesField}
+ */
+public class DoublePointFacetField extends BinaryDocValuesField {
+
+  /**
+   * Creates a new DoublePointFacetField, indexing the provided N-dimensional long point.
+   *
+   * @param name field name
+   * @param point double[] value
+   * @throws IllegalArgumentException if the field name or value is null.

Review Comment:
   Ah true I forgot, added a `null` and empty check here and in `LongPointFacetField` as well. Thanks for catching this!



##########
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. */

Review Comment:
   Changed comment to something similar to what you suggested.



##########
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:
   Yeah I think I put that there by mistake. This part of the code got deleted anyways.



##########
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:
   Changed to this `for` loop.



##########
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:
   This part of the code got removed in the next revision.



##########
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:
   It didn't even occur to me to intersect the iterators, thanks for the suggestion!



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.util.Arrays;
+import org.apache.lucene.util.NumericUtils;
+
+/** Stores a hyper rectangle as an array of DoubleRangePairs */
+public class DoubleHyperRectangle extends HyperRectangle {

Review Comment:
   Added `@lucene.experimental` tag to all new classes in the hyperrectangle package.



##########
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:
   Did not realize you could compare packed values. I think comparing packed values makes more sense here as it should be more performance than unpacking every time. Not only that but when I made the change it allowed me to simplify the code quite a bit I think.



##########
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:
   Added a note.



##########
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) */

Review Comment:
   Yeah that's correct, I forgot to remove this when I removed the enforcement.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/HyperRectangle.java:
##########
@@ -0,0 +1,101 @@
+/*
+ * 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;
+
+/** Holds the name and the number of dims for a HyperRectangle */
+public abstract class HyperRectangle {

Review Comment:
   I think we want this `public` right? Since it's a public part of the API.



##########
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:
   Changed to `shouldCountDoc`.



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/DoubleHyperRectangle.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.util.Arrays;
+import org.apache.lucene.util.NumericUtils;
+
+/** Stores a hyper rectangle as an array of DoubleRangePairs */
+public class DoubleHyperRectangle extends HyperRectangle {
+
+  /** Creates DoubleHyperRectangle */
+  public DoubleHyperRectangle(String label, DoubleRangePair... pairs) {
+    super(label, convertToLongRangePairs(pairs));
+  }
+
+  private static LongRangePair[] convertToLongRangePairs(DoubleRangePair... pairs) {
+    if (pairs == null || pairs.length == 0) {
+      throw new IllegalArgumentException("Pairs cannot be null or empty");
+    }
+    return Arrays.stream(pairs).map(DoubleRangePair::toLongRangePair).toArray(LongRangePair[]::new);
+  }
+
+  /** Defines a single range in a DoubleHyperRectangle */
+  public static class DoubleRangePair {
+    /** Inclusive min */
+    public final double min;
+
+    /** Inclusive max */
+    public final double max;
+
+    /**
+     * Creates a DoubleRangePair, very similar to the constructor of {@link
+     * org.apache.lucene.facet.range.DoubleRange}
+     *
+     * @param minIn Min value of pair
+     * @param minInclusive If minIn is inclusive
+     * @param maxIn Max value of pair
+     * @param maxInclusive If maxIn is inclusive
+     */
+    public DoubleRangePair(double minIn, boolean minInclusive, double maxIn, boolean maxInclusive) {
+      if (Double.isNaN(minIn) || Double.isNaN(maxIn)) {
+        throw new IllegalArgumentException(
+            "min and max cannot be NaN: min=" + minIn + ", max=" + maxIn);
+      }
+
+      if (!minInclusive) {
+        minIn = Math.nextUp(minIn);
+      }
+
+      if (!maxInclusive) {
+        maxIn = Math.nextDown(maxIn);
+      }
+
+      if (minIn > maxIn) {
+        throw new IllegalArgumentException(
+            "Minimum cannot be greater than maximum, max=" + maxIn + ", min=" + minIn);
+      }
+
+      this.min = minIn;
+      this.max = maxIn;
+    }
+
+    /**
+     * Converts this to a LongRangePair with sortable long equivalents
+     *
+     * @return A LongRangePair equivalent of this object
+     */
+    public LongRangePair toLongRangePair() {

Review Comment:
   Changed to `private`



##########
lucene/facet/src/java/org/apache/lucene/facet/hyperrectangle/LongPointFacetField.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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 org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.LongPoint;
+
+/** Packs an array of longs into a {@link BinaryDocValuesField} */
+public class LongPointFacetField extends BinaryDocValuesField {

Review Comment:
   I actually like your suggestion a lot, I think it makes more sense cause there is nothing really faceting specific about these fields. I will include them in the document package instead as and rename them.



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

Review Comment:
   Yeah I was thinking this would be extended later on, for example we might have a subclass that does linear scanning, another subclass that uses R trees, etc. I think I changed my mind in making things `protected` half way through writing this class though since all the functions are `private`. For now, since we aren't doing any subclassing yet, I will make it `private`.



##########
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. */

Review Comment:
   I guess that would be pronounces "Hipper rectangles" 😂. Fixed it :).



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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1156786518

   > I wonder if we shouldn't make `FacetSet` abstract and create a `LongFacetSet` for the long-specific impl
   
   I've contemplated on it when I wrote it. I like the idea of an explicit `LongFacetSet` to make it clear. The `long[]` in `FacetSet` though is used to match the dimensions. So now a user can index `IntFacetSet` and for matching construct an `FSM` with an `IntFacetSet` which translates the integers to comparable `long[]`. Then the matcher uses the read `long[]` dimension values to match to the given `comparableLong[]` ones. I'll see if I can rename it better so it's more clear.


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


[GitHub] [lucene] mdmarshmallow commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1164640793

   Yeah, I think this change should be completely compatible with 9.30. Most of our changes are isolated to the new `facetset` package and all other changes are just adding some functions to different places, which should not affect any existing functionality.


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


[GitHub] [lucene] mdmarshmallow commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
mdmarshmallow commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1159832697

   Agree with you on the `RangeMatching` solution, your way is cleaner. I'll remove it.


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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1159376910

   > It feels overly complicated to introduce `FacetSetRange<...>` and then require the different `FacetSet` implementations to implement these methods to deal with inclusive/exclusive boundaries.
   
   I agree, it's a good point. These ranges are not an attribute of a `FacetSet` and we shouldn't force custom implementations to implement that. I feel that the `fromInts/Longs/Floats/Double` gives users what they need to use `RangeFSM`.


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


[GitHub] [lucene] shaie commented on pull request #841: LUCENE-10274: Add hyperrectangle faceting capabilities

Posted by GitBox <gi...@apache.org>.
shaie commented on PR #841:
URL: https://github.com/apache/lucene/pull/841#issuecomment-1159454008

   I pushed a commit which:
   
   1. Adds demo code for a `TemperatureReading` `FacetSet` and `FacetSetMatcher`.
   2. Modifies the existing Exact and Range matcher float/double tests to properly test float/double values.
   3. Introduces default implementations of `packValues` and `sizePackedBytes` which uses `getComparableLongs`. I think this (1) clarifies why one would need to implement `getComparableLongs` and (2) provides a simple default for custom facet sets who don't care about the encoding


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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r901119495


##########
lucene/facet/src/java/org/apache/lucene/facet/facetset/MatchingFacetSetsCounts.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * 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.facetset;
+
+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.ConjunctionUtils;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Returns the counts for each given {@link FacetSet}
+ *
+ * @lucene.experimental
+ */
+public class MatchingFacetSetsCounts extends Facets {
+
+  private final FacetSetMatcher[] facetSetMatchers;
+  private final int[] counts;
+  private final String field;
+  private final int totCount;
+
+  /**
+   * Constructs a new instance of matching facet set counts which calculates the countBytes for each
+   * given facet set matcher.
+   */
+  public MatchingFacetSetsCounts(
+      String field, FacetsCollector hits, FacetSetMatcher... facetSetMatchers) throws IOException {
+    if (facetSetMatchers == null || facetSetMatchers.length == 0) {
+      throw new IllegalArgumentException("facetSetMatchers cannot be null or empty");
+    }
+    if (areFacetSetMatcherDimensionsInconsistent(facetSetMatchers)) {
+      throw new IllegalArgumentException("All facet set matchers must be the same dimensionality");
+    }
+    this.field = field;
+    this.facetSetMatchers = facetSetMatchers;
+    this.counts = new int[facetSetMatchers.length];
+    this.totCount = count(field, hits.getMatchingDocs());
+  }
+
+  /** Counts from the provided field. */
+  private int count(String field, List<FacetsCollector.MatchingDocs> matchingDocs)
+      throws IOException {
+
+    int totCount = 0;
+    for (FacetsCollector.MatchingDocs hits : matchingDocs) {
+
+      BinaryDocValues binaryDocValues = DocValues.getBinary(hits.context.reader(), field);
+
+      final DocIdSetIterator it =
+          ConjunctionUtils.intersectIterators(Arrays.asList(hits.bits.iterator(), binaryDocValues));
+      if (it == null) {
+        continue;
+      }
+
+      long[] dimValues = null; // dimension values buffer
+      int expectedNumDims = -1;
+      for (int doc = it.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = it.nextDoc()) {
+        boolean shouldCountDoc = false;
+        BytesRef bytesRef = binaryDocValues.binaryValue();
+        byte[] packedValue = bytesRef.bytes;
+        int numDims = (int) LongPoint.decodeDimension(packedValue, 0);
+        if (expectedNumDims == -1) {
+          expectedNumDims = numDims;
+          dimValues = new long[numDims];
+        } else {
+          // Verify that the number of indexed dimensions for all matching documents is the same
+          // (since we cannot verify that at indexing time).
+          assert numDims == expectedNumDims
+              : "Expected ("
+                  + expectedNumDims
+                  + ") dimensions, found ("
+                  + numDims
+                  + ") for doc ("
+                  + doc
+                  + ")";
+        }
+
+        for (int start = Long.BYTES; start < bytesRef.length; start += numDims * Long.BYTES) {
+          LongPoint.unpack(bytesRef, start, dimValues);
+          for (int j = 0; j < facetSetMatchers.length; j++) { // for each facet set matcher
+            if (facetSetMatchers[j].matches(dimValues)) {
+              counts[j]++;
+              shouldCountDoc = true;
+            }
+          }
+        }
+        if (shouldCountDoc) {
+          totCount++;
+        }
+      }
+    }
+    return totCount;
+  }
+
+  // TODO: This does not really provide "top children" functionality yet but provides "all
+  // children". This is being worked on in LUCENE-10550
+  @Override
+  public FacetResult getTopChildren(int topN, String dim, String... path) throws IOException {
+    validateTopN(topN);
+    if (!field.equals(dim)) {

Review Comment:
   Done



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


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

Posted by GitBox <gi...@apache.org>.
shaie commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_r903277577


##########
lucene/demo/src/java/org/apache/lucene/demo/facet/CustomFacetSetExample.java:
##########
@@ -0,0 +1,303 @@
+/*
+ * 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.time.LocalDate;
+import java.time.ZoneOffset;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.document.*;
+import org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.Facets;
+import org.apache.lucene.facet.FacetsCollector;
+import org.apache.lucene.facet.FacetsCollectorManager;
+import org.apache.lucene.facet.facetset.*;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.store.ByteBuffersDirectory;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * Shows usage of indexing and searching {@link FacetSetsField} with a custom {@link FacetSet}
+ * implementation. Unlike the out of the box {@link FacetSet} implementations, this example shows
+ * how to mix and match dimensions of different types, as well as implementing a custom {@link
+ * FacetSetMatcher}.
+ */
+public class CustomFacetSetExample {
+
+  private static final long MAY_SECOND_2022 = date("2022-05-02");
+  private static final long JUNE_SECOND_2022 = date("2022-06-02");
+  private static final long JULY_SECOND_2022 = date("2022-07-02");
+  private static final float HUNDRED_TWENTY_DEGREES = fahrenheitToCelsius(120);
+  private static final float HUNDRED_DEGREES = fahrenheitToCelsius(100);
+  private static final float EIGHTY_DEGREES = fahrenheitToCelsius(80);
+
+  private final Directory indexDir = new ByteBuffersDirectory();
+
+  /** Empty constructor */
+  public CustomFacetSetExample() {}
+
+  /** Build the example index. */
+  private void index() throws IOException {
+    IndexWriter indexWriter =
+        new IndexWriter(
+            indexDir, new IndexWriterConfig(new WhitespaceAnalyzer()).setOpenMode(OpenMode.CREATE));
+
+    // Every document holds the temperature measures for a City by Date
+
+    Document doc = new Document();
+    doc.add(new StringField("city", "city1", Field.Store.YES));
+    doc.add(
+        FacetSetsField.create(
+            "temperature",
+            new TemperatureReadingFacetSet(MAY_SECOND_2022, HUNDRED_DEGREES),
+            new TemperatureReadingFacetSet(JUNE_SECOND_2022, EIGHTY_DEGREES),
+            new TemperatureReadingFacetSet(JULY_SECOND_2022, HUNDRED_TWENTY_DEGREES)));
+    indexWriter.addDocument(doc);
+
+    doc = new Document();
+    doc.add(new StringField("city", "city2", Field.Store.YES));
+    doc.add(
+        FacetSetsField.create(
+            "temperature",
+            new TemperatureReadingFacetSet(MAY_SECOND_2022, EIGHTY_DEGREES),
+            new TemperatureReadingFacetSet(JUNE_SECOND_2022, HUNDRED_DEGREES),
+            new TemperatureReadingFacetSet(JULY_SECOND_2022, HUNDRED_TWENTY_DEGREES)));
+    indexWriter.addDocument(doc);
+
+    indexWriter.close();
+  }
+
+  /** Counting documents which exactly match a given {@link FacetSet}. */
+  private List<FacetResult> exactMatching() throws IOException {
+    DirectoryReader indexReader = DirectoryReader.open(indexDir);
+    IndexSearcher searcher = new IndexSearcher(indexReader);
+
+    // MatchAllDocsQuery is for "browsing" (counts facets
+    // for all non-deleted docs in the index); normally
+    // you'd use a "normal" query:
+    FacetsCollector fc = searcher.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    // Count both "Publish Date" and "Author" dimensions
+    Facets facets =
+        new MatchingFacetSetsCounts(
+            "temperature",
+            fc,
+            TemperatureReadingFacetSet::decodeTemperatureReading,
+            new ExactFacetSetMatcher(
+                "May 2022 (100f)",
+                new TemperatureReadingFacetSet(MAY_SECOND_2022, HUNDRED_DEGREES)),
+            new ExactFacetSetMatcher(
+                "July 2022 (120f)",
+                new TemperatureReadingFacetSet(JULY_SECOND_2022, HUNDRED_TWENTY_DEGREES)));
+
+    // Retrieve results
+    List<FacetResult> results = Collections.singletonList(facets.getTopChildren(10, "temperature"));

Review Comment:
   It's the same "oops" from above :)



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