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/14 19:12:46 UTC

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

gsmiller commented on code in PR #841:
URL: https://github.com/apache/lucene/pull/841#discussion_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