You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2022/06/25 15:13:16 UTC

[lucene] branch branch_9x updated: LUCENE-10274: Add FacetSets faceting capabilities (#841) (#981)

This is an automated email from the ASF dual-hosted git repository.

shaie pushed a commit to branch branch_9x
in repository https://gitbox.apache.org/repos/asf/lucene.git


The following commit(s) were added to refs/heads/branch_9x by this push:
     new 0e1502c12ba LUCENE-10274: Add FacetSets faceting capabilities (#841) (#981)
0e1502c12ba is described below

commit 0e1502c12ba93ce42e608a93ab71314a6d671a13
Author: Shai Erera <se...@gmail.com>
AuthorDate: Sat Jun 25 18:13:12 2022 +0300

    LUCENE-10274: Add FacetSets faceting capabilities (#841) (#981)
    
    * LUCENE-10274: Add FacetSets faceting capabilities (#841)
    
    Co-authored-by: Marc D'Mello <dm...@amazon.com>
    Co-authored-by: Shai Erera <se...@gmail.com>
    Co-authored-by: Greg Miller <gs...@gmail.com>
---
 lucene/CHANGES.txt                                 |   3 +
 .../java/org/apache/lucene/document/LongPoint.java |  19 +
 .../lucene/demo/facet/CustomFacetSetExample.java   | 301 +++++++++++++++
 .../demo/facet/TestCustomFacetSetExample.java      |  62 +++
 lucene/facet/docs/FacetSets.adoc                   | 132 +++++++
 lucene/facet/src/java/module-info.java             |   1 +
 .../org/apache/lucene/facet/facetset/DimRange.java | 119 ++++++
 .../lucene/facet/facetset/DoubleFacetSet.java      |  50 +++
 .../facet/facetset/ExactFacetSetMatcher.java       |  49 +++
 .../org/apache/lucene/facet/facetset/FacetSet.java |  62 +++
 .../lucene/facet/facetset/FacetSetDecoder.java     |  59 +++
 .../lucene/facet/facetset/FacetSetMatcher.java     |  55 +++
 .../lucene/facet/facetset/FacetSetsField.java      |  78 ++++
 .../lucene/facet/facetset/FloatFacetSet.java       |  66 ++++
 .../apache/lucene/facet/facetset/IntFacetSet.java  |  63 ++++
 .../lucene/facet/facetset/LongFacetSet.java}       |  39 +-
 .../facet/facetset/MatchingFacetSetsCounts.java    | 164 ++++++++
 .../facet/facetset/RangeFacetSetMatcher.java       |  69 ++++
 .../lucene/facet/facetset/package-info.java}       |  18 +-
 .../facet/facetset/TestExactFacetSetMatcher.java   | 333 ++++++++++++++++
 .../facetset/TestMatchingFacetSetsCounts.java      |  90 +++++
 .../facet/facetset/TestRangeFacetSetMatcher.java   | 418 +++++++++++++++++++++
 22 files changed, 2226 insertions(+), 24 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 658c3ee539f..9279c266c02 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -77,6 +77,9 @@ New Features
 ---------------------
 * LUCENE-10550: Add getAllChildren functionality to facets (Yuting Gan)
 
+* LUCENE-10274: Added facetsets module for high dimensional (hyper-rectangle) faceting
+(Shai Erera, Marc D'Mello, Greg Miller)
+
 Improvements
 ---------------------
 
diff --git a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
index ae9438697bd..2417f22fcf3 100644
--- a/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
+++ b/lucene/core/src/java/org/apache/lucene/document/LongPoint.java
@@ -117,6 +117,25 @@ public final class LongPoint extends Field {
     return new BytesRef(packed);
   }
 
+  /**
+   * Unpack a BytesRef into a long point. This method can be used to unpack values that were packed
+   * with {@link #pack(long...)}.
+   *
+   * @param bytesRef BytesRef Value
+   * @param start the start offset to unpack the values from
+   * @param buf the buffer to store the values in
+   * @throws IllegalArgumentException if bytesRef or buf are null
+   */
+  public static void unpack(BytesRef bytesRef, int start, long[] buf) {
+    if (bytesRef == null || buf == null) {
+      throw new IllegalArgumentException("bytesRef and buf must not be null");
+    }
+
+    for (int i = 0, offset = start; i < buf.length; i++, offset += Long.BYTES) {
+      buf[i] = LongPoint.decodeDimension(bytesRef.bytes, offset);
+    }
+  }
+
   /**
    * Creates a new LongPoint, indexing the provided N-dimensional long point.
    *
diff --git a/lucene/demo/src/java/org/apache/lucene/demo/facet/CustomFacetSetExample.java b/lucene/demo/src/java/org/apache/lucene/demo/facet/CustomFacetSetExample.java
new file mode 100644
index 00000000000..c688c17d17f
--- /dev/null
+++ b/lucene/demo/src/java/org/apache/lucene/demo/facet/CustomFacetSetExample.java
@@ -0,0 +1,301 @@
+/*
+ * 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 org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.document.StringField;
+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.DimRange;
+import org.apache.lucene.facet.facetset.ExactFacetSetMatcher;
+import org.apache.lucene.facet.facetset.FacetSet;
+import org.apache.lucene.facet.facetset.FacetSetDecoder;
+import org.apache.lucene.facet.facetset.FacetSetMatcher;
+import org.apache.lucene.facet.facetset.FacetSetsField;
+import org.apache.lucene.facet.facetset.MatchingFacetSetsCounts;
+import org.apache.lucene.facet.facetset.RangeFacetSetMatcher;
+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 FacetResult exactMatching() throws IOException {
+    try (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 "May 2022, 100 degrees" and "July 2022, 120 degrees" 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
+      return facets.getAllChildren("temperature");
+    }
+  }
+
+  /** Counting documents which match a certain degrees value for any date. */
+  private FacetResult rangeMatching() throws IOException {
+    try (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 80-100 degrees
+      Facets facets =
+          new MatchingFacetSetsCounts(
+              "temperature",
+              fc,
+              TemperatureReadingFacetSet::decodeTemperatureReading,
+              new RangeFacetSetMatcher(
+                  "Eighty to Hundred Degrees",
+                  DimRange.fromLongs(Long.MIN_VALUE, true, Long.MAX_VALUE, true),
+                  DimRange.fromFloats(EIGHTY_DEGREES, true, HUNDRED_DEGREES, true)));
+
+      // Retrieve results
+      return facets.getAllChildren("temperature");
+    }
+  }
+
+  /**
+   * Like {@link #rangeMatching()}, however this example demonstrates a custom {@link
+   * FacetSetMatcher} which only considers certain dimensions (in this case only the temperature
+   * one).
+   */
+  private FacetResult customRangeMatching() throws IOException {
+    try (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 80-100 degrees
+      Facets facets =
+          new MatchingFacetSetsCounts(
+              "temperature",
+              fc,
+              TemperatureReadingFacetSet::decodeTemperatureReading,
+              new TemperatureOnlyFacetSetMatcher(
+                  "Eighty to Hundred Degrees",
+                  DimRange.fromFloats(EIGHTY_DEGREES, true, HUNDRED_DEGREES, true)));
+
+      // Retrieve results
+      return facets.getAllChildren("temperature");
+    }
+  }
+
+  private static long date(String dateString) {
+    return LocalDate.parse(dateString).atStartOfDay().toInstant(ZoneOffset.UTC).toEpochMilli();
+  }
+
+  private static float fahrenheitToCelsius(int degrees) {
+    return (degrees - 32.0f) * 5.f / 9.f;
+  }
+
+  /** Runs the exact matching example. */
+  public FacetResult runExactMatching() throws IOException {
+    index();
+    return exactMatching();
+  }
+
+  /** Runs the range matching example. */
+  public FacetResult runRangeMatching() throws IOException {
+    index();
+    return rangeMatching();
+  }
+
+  /** Runs the custom range matching example. */
+  public FacetResult runCustomRangeMatching() throws IOException {
+    index();
+    return customRangeMatching();
+  }
+
+  /** Runs the search and drill-down examples and prints the results. */
+  public static void main(String[] args) throws Exception {
+    CustomFacetSetExample example = new CustomFacetSetExample();
+
+    System.out.println("Exact Facet Set matching example:");
+    System.out.println("-----------------------");
+    FacetResult result = example.runExactMatching();
+    System.out.println("Temperature Reading: " + result);
+
+    System.out.println("Range Facet Set matching example:");
+    System.out.println("-----------------------");
+    result = example.runRangeMatching();
+    System.out.println("Temperature Reading: " + result);
+
+    System.out.println("Custom Range Facet Set matching example:");
+    System.out.println("-----------------------");
+    result = example.runCustomRangeMatching();
+    System.out.println("Temperature Reading: " + result);
+  }
+
+  /**
+   * A {@link FacetSet} which encodes a temperature reading in a date (long) and degrees (celsius;
+   * float).
+   */
+  public static class TemperatureReadingFacetSet extends FacetSet {
+
+    private static final int SIZE_PACKED_BYTES = Long.BYTES + Float.BYTES;
+
+    private final long date;
+    private final float degrees;
+
+    /** Constructor */
+    public TemperatureReadingFacetSet(long date, float degrees) {
+      super(2); // We encode two dimensions
+
+      this.date = date;
+      this.degrees = degrees;
+    }
+
+    @Override
+    public long[] getComparableValues() {
+      return new long[] {date, NumericUtils.floatToSortableInt(degrees)};
+    }
+
+    @Override
+    public int packValues(byte[] buf, int start) {
+      LongPoint.encodeDimension(date, buf, start);
+      // Encode 'degrees' as a sortable integer.
+      FloatPoint.encodeDimension(degrees, buf, start + Long.BYTES);
+      return sizePackedBytes();
+    }
+
+    @Override
+    public int sizePackedBytes() {
+      return SIZE_PACKED_BYTES;
+    }
+
+    /**
+     * An implementation of {@link FacetSetDecoder#decode(BytesRef, int, long[])} for {@link
+     * TemperatureReadingFacetSet}.
+     */
+    public static int decodeTemperatureReading(BytesRef bytesRef, int start, long[] dest) {
+      dest[0] = LongPoint.decodeDimension(bytesRef.bytes, start);
+      // Decode the degrees as a sortable integer.
+      dest[1] = IntPoint.decodeDimension(bytesRef.bytes, start + Long.BYTES);
+      return SIZE_PACKED_BYTES;
+    }
+  }
+
+  /**
+   * A {@link FacetSetMatcher} which matches facet sets only by their temperature dimension,
+   * ignoring the date.
+   */
+  public static class TemperatureOnlyFacetSetMatcher extends FacetSetMatcher {
+
+    private final DimRange temperatureRange;
+
+    /** Constructor */
+    protected TemperatureOnlyFacetSetMatcher(String label, DimRange temperatureRange) {
+      super(label, 1); // We only evaluate one dimension
+
+      this.temperatureRange = temperatureRange;
+    }
+
+    @Override
+    public boolean matches(long[] dimValues) {
+      return temperatureRange.min <= dimValues[1] && temperatureRange.max >= dimValues[1];
+    }
+  }
+}
diff --git a/lucene/demo/src/test/org/apache/lucene/demo/facet/TestCustomFacetSetExample.java b/lucene/demo/src/test/org/apache/lucene/demo/facet/TestCustomFacetSetExample.java
new file mode 100644
index 00000000000..31bacffa51c
--- /dev/null
+++ b/lucene/demo/src/test/org/apache/lucene/demo/facet/TestCustomFacetSetExample.java
@@ -0,0 +1,62 @@
+/*
+ * 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 org.apache.lucene.facet.FacetResult;
+import org.apache.lucene.facet.LabelAndValue;
+import org.apache.lucene.tests.util.LuceneTestCase;
+import org.junit.Test;
+
+public class TestCustomFacetSetExample extends LuceneTestCase {
+
+  @Test
+  public void testExactMatching() throws Exception {
+    FacetResult result = new CustomFacetSetExample().runExactMatching();
+
+    assertEquals("temperature", result.dim);
+    assertEquals(0, result.path.length);
+    assertEquals(2, result.value);
+    assertEquals(2, result.childCount);
+
+    assertEquals(new LabelAndValue("May 2022 (100f)", 1), result.labelValues[0]);
+    assertEquals(new LabelAndValue("July 2022 (120f)", 2), result.labelValues[1]);
+  }
+
+  @Test
+  public void testRangeMatching() throws Exception {
+    FacetResult result = new CustomFacetSetExample().runRangeMatching();
+
+    assertEquals("temperature", result.dim);
+    assertEquals(0, result.path.length);
+    assertEquals(2, result.value);
+    assertEquals(1, result.childCount);
+
+    assertEquals(new LabelAndValue("Eighty to Hundred Degrees", 4), result.labelValues[0]);
+  }
+
+  @Test
+  public void testCustomRangeMatching() throws Exception {
+    FacetResult result = new CustomFacetSetExample().runCustomRangeMatching();
+
+    assertEquals("temperature", result.dim);
+    assertEquals(0, result.path.length);
+    assertEquals(2, result.value);
+    assertEquals(1, result.childCount);
+
+    assertEquals(new LabelAndValue("Eighty to Hundred Degrees", 4), result.labelValues[0]);
+  }
+}
diff --git a/lucene/facet/docs/FacetSets.adoc b/lucene/facet/docs/FacetSets.adoc
new file mode 100644
index 00000000000..90ad52d7d93
--- /dev/null
+++ b/lucene/facet/docs/FacetSets.adoc
@@ -0,0 +1,132 @@
+= 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
+
+A functional interface for decoding the packed dimension values back into comparable `long[]` representation. The
+off-the-shelf `Int/FloatFacetSet` can use `FacetSetDecoder::decodeInts` and `Long/DoubleFacetSet` can use
+`FacetSetDecoder::decodeLongs` to decode the indexed dimension values. You can check out
+`org.apache.lucene.demo.facet.CustomFacetSetExample` in the `lucene/demo` package for a custom implementation of a
+`FacetSetDecoder`.
+
+=== MatchingFacetSetCounts
+
+A `Facets` implementation which counts how many documents are matched by each of a list of `FacetSetMatcher`, in
+addition to counting the total number of documents that were matched. It uses the `FacetSetDecoder` to decode the
+dimension values.
+
+== FacetSets Under the Hood
+
+Facet sets are indexed as a `BinaryDocValues` field where the dimension values are encoded as a `byte[]` which consists
+of the number of dimensions in a set (`int`) followed by the packed values of each set (as implemented by
+`packValues()`). At aggregation time the values are decoded by `FacetSetDecoder` into a `long[]` and the matching of the
+values occurs in the `long` space.
+
+All out of the box `FacetSet` implementations pack the dimension values as sortable `long` (`Long/DoubleFacetSet`) or
+`int` (`Int/FloatFacetSet`). If you implement your own `FacetSet` and choose to encode the values differently (e.g. by
+compressing them), you should provide a corresponding implementation of a `FacetSetDecoder`.
diff --git a/lucene/facet/src/java/module-info.java b/lucene/facet/src/java/module-info.java
index 12ebd3ef37d..6810bf43019 100644
--- a/lucene/facet/src/java/module-info.java
+++ b/lucene/facet/src/java/module-info.java
@@ -27,4 +27,5 @@ module org.apache.lucene.facet {
   exports org.apache.lucene.facet.taxonomy;
   exports org.apache.lucene.facet.taxonomy.directory;
   exports org.apache.lucene.facet.taxonomy.writercache;
+  exports org.apache.lucene.facet.facetset;
 }
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/facetset/DimRange.java b/lucene/facet/src/java/org/apache/lucene/facet/facetset/DimRange.java
new file mode 100644
index 00000000000..777e91ad6c2
--- /dev/null
+++ b/lucene/facet/src/java/org/apache/lucene/facet/facetset/DimRange.java
@@ -0,0 +1,119 @@
+/*
+ * 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.util.NumericUtils;
+
+/**
+ * Defines a single range in a {@link FacetSet} dimension.
+ *
+ * @lucene.experimental
+ */
+public class DimRange {
+  /** Inclusive min */
+  public final long min;
+
+  /** Inclusive max */
+  public final long max;
+
+  /**
+   * Creates a LongRange.
+   *
+   * @param min inclusive min value in range
+   * @param max inclusive max value in range
+   */
+  public DimRange(long min, long max) {
+    this.min = min;
+    this.max = max;
+  }
+
+  /**
+   * 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) {
+    if (!minInclusive) {
+      if (min != Long.MAX_VALUE) {
+        min++;
+      } else {
+        throw new IllegalArgumentException("Invalid min input: " + min);
+      }
+    }
+
+    if (!maxInclusive) {
+      if (max != Long.MIN_VALUE) {
+        max--;
+      } else {
+        throw new IllegalArgumentException("Invalid max input: " + max);
+      }
+    }
+
+    if (min > max) {
+      throw new IllegalArgumentException(
+          "Minimum cannot be greater than maximum, max=" + max + ", min=" + min);
+    }
+
+    return new DimRange(min, max);
+  }
+
+  /** Creates a {@link DimRange} for the given min and max double values. */
+  public static DimRange fromDoubles(
+      double min, boolean minInclusive, double max, boolean maxInclusive) {
+    if (Double.isNaN(min)) {
+      throw new IllegalArgumentException("min cannot be NaN");
+    }
+    if (!minInclusive) {
+      min = Math.nextUp(min);
+    }
+
+    if (Double.isNaN(max)) {
+      throw new IllegalArgumentException("max cannot be NaN");
+    }
+    if (!maxInclusive) {
+      max = Math.nextDown(max);
+    }
+
+    if (min > max) {
+      throw new IllegalArgumentException("Minimum cannot be greater than maximum");
+    }
+    return new DimRange(
+        NumericUtils.doubleToSortableLong(min), NumericUtils.doubleToSortableLong(max));
+  }
+
+  /** Creates a {@link DimRange} for the given min and max float values. */
+  public static DimRange fromFloats(
+      float min, boolean minInclusive, float max, boolean maxInclusive) {
+    if (Float.isNaN(min)) {
+      throw new IllegalArgumentException("min cannot be NaN");
+    }
+    if (!minInclusive) {
+      min = Math.nextUp(min);
+    }
+
+    if (Float.isNaN(max)) {
+      throw new IllegalArgumentException("max cannot be NaN");
+    }
+    if (!maxInclusive) {
+      max = Math.nextDown(max);
+    }
+
+    if (min > max) {
+      throw new IllegalArgumentException("Minimum cannot be greater than maximum");
+    }
+    return new DimRange(NumericUtils.floatToSortableInt(min), NumericUtils.floatToSortableInt(max));
+  }
+}
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/facetset/DoubleFacetSet.java b/lucene/facet/src/java/org/apache/lucene/facet/facetset/DoubleFacetSet.java
new file mode 100644
index 00000000000..826faf959f1
--- /dev/null
+++ b/lucene/facet/src/java/org/apache/lucene/facet/facetset/DoubleFacetSet.java
@@ -0,0 +1,50 @@
+/*
+ * 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 FacetSet} which encodes double dimension values.
+ *
+ * @lucene.experimental
+ */
+public class DoubleFacetSet extends FacetSet {
+
+  /** The raw dimension values of this facet set. */
+  public final double[] values;
+
+  /** Constructs a new instance of a facet set which stores {@code double} dimension values. */
+  public DoubleFacetSet(double... values) {
+    super(validateValuesAndGetNumDims(values));
+
+    this.values = values;
+  }
+
+  @Override
+  public long[] getComparableValues() {
+    return Arrays.stream(values).mapToLong(NumericUtils::doubleToSortableLong).toArray();
+  }
+
+  private static int validateValuesAndGetNumDims(double... values) {
+    if (values == null || values.length == 0) {
+      throw new IllegalArgumentException("values cannot be null or empty");
+    }
+    return values.length;
+  }
+}
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/facetset/ExactFacetSetMatcher.java b/lucene/facet/src/java/org/apache/lucene/facet/facetset/ExactFacetSetMatcher.java
new file mode 100644
index 00000000000..b3b9b40f478
--- /dev/null
+++ b/lucene/facet/src/java/org/apache/lucene/facet/facetset/ExactFacetSetMatcher.java
@@ -0,0 +1,49 @@
+/*
+ * 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 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.dims);
+
+    this.values = facetSet.getComparableValues();
+  }
+
+  @Override
+  public boolean matches(long[] dimValues) {
+    assert dimValues.length == dims
+        : "Encoded dimensions (dims="
+            + dimValues.length
+            + ") is incompatible with FacetSet dimensions (dims="
+            + dims
+            + ")";
+
+    return Arrays.equals(dimValues, values);
+  }
+}
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSet.java b/lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSet.java
new file mode 100644
index 00000000000..4fd047f1a4d
--- /dev/null
+++ b/lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSet.java
@@ -0,0 +1,62 @@
+/*
+ * 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.LongPoint;
+
+/**
+ * Holds a set of facet dimension values.
+ *
+ * @lucene.experimental
+ */
+public abstract class FacetSet {
+
+  /** The number of dimension values in this set. */
+  public final int dims;
+
+  /** Constructs a new instance of a facet set with the given number of dimensions. */
+  protected FacetSet(int dims) {
+    this.dims = dims;
+  }
+
+  /** Returns the dimension values in this facet set as comparable longs. */
+  public abstract long[] getComparableValues();
+
+  /**
+   * Packs the dimension values into the given {@code byte[]} and returns the number of
+   * packed-values bytes. The default implementation packs the {@link #getComparableValues()
+   * comparable values}, and you can override to implement your own scheme.
+   */
+  public int packValues(byte[] buf, int start) {
+    long[] comparableValues = getComparableValues();
+    for (int i = 0, offset = start; i < comparableValues.length; i++, offset += Long.BYTES) {
+      LongPoint.encodeDimension(comparableValues[i], buf, offset);
+    }
+    return comparableValues.length * Long.BYTES;
+  }
+
+  /**
+   * Returns the size of the packed values in this facet set. If the value is unknown in advance
+   * (e.g. if the values are compressed), this method can return an upper limit. The default
+   * implementations returns {@code dims * Long.BYTES} per the default implementation of {@link
+   * #packValues(byte[], int)}. You should override if you implement {@link #packValues(byte[],
+   * int)} differently.
+   */
+  public int sizePackedBytes() {
+    return dims * Long.BYTES;
+  }
+}
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSetDecoder.java b/lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSetDecoder.java
new file mode 100644
index 00000000000..008f9d72f34
--- /dev/null
+++ b/lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSetDecoder.java
@@ -0,0 +1,59 @@
+/*
+ * 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.IntPoint;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A functional interface for decoding facet set values into comparable `long` ones. You can use it
+ * by e.g. implementing a static method with the same signature and then pass it as {@code
+ * YourClass::decode}.
+ *
+ * @lucene.experimental
+ */
+public interface FacetSetDecoder {
+
+  /**
+   * An implementation of {@link FacetSetDecoder#decode(BytesRef, int, long[])} for long/double
+   * dimension values which were encoded with {@link LongFacetSet} and {@link DoubleFacetSet}
+   * respectively.
+   */
+  static int decodeLongs(BytesRef bytesRef, int start, long[] dest) {
+    LongPoint.unpack(bytesRef, start, dest);
+    return dest.length * Long.BYTES;
+  }
+
+  /**
+   * An implementation of {@link FacetSetDecoder#decode(BytesRef, int, long[])} for int/float
+   * dimension values which were encoded with {@link IntFacetSet} and {@link FloatFacetSet}
+   * respectively.
+   */
+  static int decodeInts(BytesRef bytesRef, int start, long[] dest) {
+    for (int i = 0, offset = start; i < dest.length; i++, offset += Integer.BYTES) {
+      dest[i] = IntPoint.decodeDimension(bytesRef.bytes, offset);
+    }
+    return dest.length * Integer.BYTES;
+  }
+
+  /**
+   * Decodes the facet set dimension values into the given destination buffer and returns the number
+   * of bytes read.
+   */
+  int decode(BytesRef bytesRef, int start, long[] dest);
+}
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSetMatcher.java b/lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSetMatcher.java
new file mode 100644
index 00000000000..495cb4e6a5c
--- /dev/null
+++ b/lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSetMatcher.java
@@ -0,0 +1,55 @@
+/*
+ * 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;
+
+/**
+ * Matches the encoded {@link FacetSet} that was indexed in {@link FacetSetsField}.
+ *
+ * @lucene.experimental
+ */
+public abstract class FacetSetMatcher {
+
+  /** The label to associate to this matcher's aggregated value. */
+  public final String label;
+
+  /** The number of dimensions that are matched by this matcher. */
+  public final int dims;
+
+  /**
+   * Constructs a new matcher.
+   *
+   * @param label the label to associate to this matcher's aggregated value
+   * @param dims the number of dimensions that are matched by this matcher
+   */
+  protected FacetSetMatcher(String label, int dims) {
+    if (label == null) {
+      throw new IllegalArgumentException("label cannot be null");
+    }
+    if (dims <= 0) {
+      throw new IllegalArgumentException("dims must be greater than 0: " + dims);
+    }
+    this.label = label;
+    this.dims = dims;
+  }
+
+  /**
+   * Returns true if the facet set encoded in the given {@code long[]} is matched by this matcher.
+   *
+   * @param dimValues the encoded dimension values of a single facet set in this field
+   */
+  public abstract boolean matches(long[] dimValues);
+}
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSetsField.java b/lucene/facet/src/java/org/apache/lucene/facet/facetset/FacetSetsField.java
new file mode 100644
index 00000000000..b2f2db7bb1f
--- /dev/null
+++ b/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) {
+        throw new IllegalArgumentException(
+            "All FacetSets must be the same type. Expected "
+                + expectedClass
+                + " found "
+                + facetSet.getClass());
+      }
+      offset += facetSet.packValues(buf, offset);
+    }
+    return new BytesRef(buf, 0, offset);
+  }
+}
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/facetset/FloatFacetSet.java b/lucene/facet/src/java/org/apache/lucene/facet/facetset/FloatFacetSet.java
new file mode 100644
index 00000000000..05f5f4c4a9c
--- /dev/null
+++ b/lucene/facet/src/java/org/apache/lucene/facet/facetset/FloatFacetSet.java
@@ -0,0 +1,66 @@
+/*
+ * 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.
+ *
+ * @lucene.experimental
+ */
+public class FloatFacetSet extends FacetSet {
+
+  /** The raw dimension values of this facet set. */
+  public final float[] values;
+
+  /** Constructs a new instance of a facet set which stores {@code float} dimension values. */
+  public FloatFacetSet(float... values) {
+    super(validateValuesAndGetNumDims(values));
+
+    this.values = values;
+  }
+
+  @Override
+  public long[] getComparableValues() {
+    return IntStream.range(0, values.length)
+        .mapToLong(idx -> NumericUtils.floatToSortableInt(values[idx]))
+        .toArray();
+  }
+
+  @Override
+  public int packValues(byte[] buf, int start) {
+    for (int i = 0, offset = start; i < values.length; i++, offset += Float.BYTES) {
+      FloatPoint.encodeDimension(values[i], buf, offset);
+    }
+    return values.length * Float.BYTES;
+  }
+
+  @Override
+  public int sizePackedBytes() {
+    return dims * Float.BYTES;
+  }
+
+  private static int validateValuesAndGetNumDims(float... values) {
+    if (values == null || values.length == 0) {
+      throw new IllegalArgumentException("values cannot be null or empty");
+    }
+    return values.length;
+  }
+}
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/facetset/IntFacetSet.java b/lucene/facet/src/java/org/apache/lucene/facet/facetset/IntFacetSet.java
new file mode 100644
index 00000000000..13b78808f5e
--- /dev/null
+++ b/lucene/facet/src/java/org/apache/lucene/facet/facetset/IntFacetSet.java
@@ -0,0 +1,63 @@
+/*
+ * 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.IntPoint;
+
+/**
+ * A {@link FacetSet} which encodes integer dimension values.
+ *
+ * @lucene.experimental
+ */
+public class IntFacetSet extends FacetSet {
+
+  /** The raw dimension values of this facet set. */
+  public final int[] values;
+
+  /** Constructs a new instance of a facet set which stores {@code int} dimension values. */
+  public IntFacetSet(int... values) {
+    super(validateValuesAndGetNumDims(values));
+
+    this.values = values;
+  }
+
+  @Override
+  public long[] getComparableValues() {
+    return Arrays.stream(values).mapToLong(Long::valueOf).toArray();
+  }
+
+  @Override
+  public int packValues(byte[] buf, int start) {
+    for (int i = 0, offset = start; i < values.length; i++, offset += Integer.BYTES) {
+      IntPoint.encodeDimension(values[i], buf, offset);
+    }
+    return values.length * Integer.BYTES;
+  }
+
+  @Override
+  public int sizePackedBytes() {
+    return dims * Integer.BYTES;
+  }
+
+  private static int validateValuesAndGetNumDims(int... values) {
+    if (values == null || values.length == 0) {
+      throw new IllegalArgumentException("values cannot be null or empty");
+    }
+    return values.length;
+  }
+}
diff --git a/lucene/facet/src/java/module-info.java b/lucene/facet/src/java/org/apache/lucene/facet/facetset/LongFacetSet.java
similarity index 50%
copy from lucene/facet/src/java/module-info.java
copy to lucene/facet/src/java/org/apache/lucene/facet/facetset/LongFacetSet.java
index 12ebd3ef37d..723be7cc390 100644
--- a/lucene/facet/src/java/module-info.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/facetset/LongFacetSet.java
@@ -14,17 +14,34 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.lucene.facet.facetset;
 
-/** Faceted indexing and search capabilities */
-@SuppressWarnings({"requires-automatic"})
-module org.apache.lucene.facet {
-  requires com.carrotsearch.hppc;
-  requires org.apache.lucene.core;
+/**
+ * A {@link FacetSet} which encodes long dimension values.
+ *
+ * @lucene.experimental
+ */
+public class LongFacetSet extends FacetSet {
+
+  /** The raw dimension values of this facet set. */
+  public final long[] values;
+
+  /** Constructs a new instance of a facet set which stores {@code long} dimension values. */
+  public LongFacetSet(long... values) {
+    super(validateValuesAndGetNumDims(values));
+
+    this.values = values;
+  }
+
+  @Override
+  public long[] getComparableValues() {
+    return values;
+  }
 
-  exports org.apache.lucene.facet;
-  exports org.apache.lucene.facet.range;
-  exports org.apache.lucene.facet.sortedset;
-  exports org.apache.lucene.facet.taxonomy;
-  exports org.apache.lucene.facet.taxonomy.directory;
-  exports org.apache.lucene.facet.taxonomy.writercache;
+  private static int validateValuesAndGetNumDims(long... values) {
+    if (values == null || values.length == 0) {
+      throw new IllegalArgumentException("values cannot be null or empty");
+    }
+    return values.length;
+  }
 }
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/facetset/MatchingFacetSetsCounts.java b/lucene/facet/src/java/org/apache/lucene/facet/facetset/MatchingFacetSetsCounts.java
new file mode 100644
index 00000000000..07a61d69e53
--- /dev/null
+++ b/lucene/facet/src/java/org/apache/lucene/facet/facetset/MatchingFacetSetsCounts.java
@@ -0,0 +1,164 @@
+/*
+ * 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.IntPoint;
+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 FacetSetDecoder facetSetDecoder;
+  private final int totCount;
+
+  /**
+   * Constructs a new instance of matching facet set counts which calculates the counts for each
+   * given facet set matcher.
+   */
+  public MatchingFacetSetsCounts(
+      String field,
+      FacetsCollector hits,
+      FacetSetDecoder facetSetDecoder,
+      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.facetSetDecoder = facetSetDecoder;
+    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 = IntPoint.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 = Integer.BYTES; start < bytesRef.length; ) {
+          start += facetSetDecoder.decode(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;
+  }
+
+  @Override
+  public FacetResult getAllChildren(String dim, String... path) throws IOException {
+    if (field.equals(dim) == false) {
+      throw new IllegalArgumentException(
+          "invalid dim \"" + dim + "\"; should be \"" + field + "\"");
+    }
+    if (path != null && 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(facetSetMatchers[i].label, counts[i]);
+    }
+    return new FacetResult(dim, path, totCount, labelValues, labelValues.length);
+  }
+
+  @Override
+  public FacetResult getTopChildren(int topN, String dim, String... path) throws IOException {
+    validateTopN(topN);
+    return getAllChildren(dim, path);
+  }
+
+  @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));
+  }
+
+  private static boolean areFacetSetMatcherDimensionsInconsistent(
+      FacetSetMatcher[] facetSetMatchers) {
+    int dims = facetSetMatchers[0].dims;
+    return Arrays.stream(facetSetMatchers)
+        .anyMatch(facetSetMatcher -> facetSetMatcher.dims != dims);
+  }
+}
diff --git a/lucene/facet/src/java/org/apache/lucene/facet/facetset/RangeFacetSetMatcher.java b/lucene/facet/src/java/org/apache/lucene/facet/facetset/RangeFacetSetMatcher.java
new file mode 100644
index 00000000000..9ed12e0b6b2
--- /dev/null
+++ b/lucene/facet/src/java/org/apache/lucene/facet/facetset/RangeFacetSetMatcher.java
@@ -0,0 +1,69 @@
+/*
+ * 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 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;
+  }
+}
diff --git a/lucene/facet/src/java/module-info.java b/lucene/facet/src/java/org/apache/lucene/facet/facetset/package-info.java
similarity index 63%
copy from lucene/facet/src/java/module-info.java
copy to lucene/facet/src/java/org/apache/lucene/facet/facetset/package-info.java
index 12ebd3ef37d..d69d251573d 100644
--- a/lucene/facet/src/java/module-info.java
+++ b/lucene/facet/src/java/org/apache/lucene/facet/facetset/package-info.java
@@ -15,16 +15,8 @@
  * limitations under the License.
  */
 
-/** Faceted indexing and search capabilities */
-@SuppressWarnings({"requires-automatic"})
-module org.apache.lucene.facet {
-  requires com.carrotsearch.hppc;
-  requires org.apache.lucene.core;
-
-  exports org.apache.lucene.facet;
-  exports org.apache.lucene.facet.range;
-  exports org.apache.lucene.facet.sortedset;
-  exports org.apache.lucene.facet.taxonomy;
-  exports org.apache.lucene.facet.taxonomy.directory;
-  exports org.apache.lucene.facet.taxonomy.writercache;
-}
+/**
+ * Provides FacetSets faceting capabilities which allows users to facet on high dimensional field
+ * values. See FacetSets.adoc in the docs folder for more information on usage.
+ */
+package org.apache.lucene.facet.facetset;
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/facetset/TestExactFacetSetMatcher.java b/lucene/facet/src/test/org/apache/lucene/facet/facetset/TestExactFacetSetMatcher.java
new file mode 100644
index 00000000000..9751822273a
--- /dev/null
+++ b/lucene/facet/src/test/org/apache/lucene/facet/facetset/TestExactFacetSetMatcher.java
@@ -0,0 +1,333 @@
+/*
+ * 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.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.facet.*;
+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;
+import org.apache.lucene.tests.util.TestUtil;
+
+public class TestExactFacetSetMatcher extends FacetTestCase {
+
+  private static final int FORD_ORD = 100;
+  private static final int TOYOTA_ORD = 101;
+  private static final int CHEVY_ORD = 102;
+  private static final int NISSAN_ORD = 103;
+  private static final int[] MANUFACTURER_ORDS = {FORD_ORD, TOYOTA_ORD, CHEVY_ORD, NISSAN_ORD};
+  private static final int[] YEARS = {2010, 2011, 2012};
+
+  public void testSimpleFacetSetMatching() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+
+    // Ford-2010, Chevy-2011
+    Document doc = new Document();
+    doc.add(
+        FacetSetsField.create(
+            "field", new LongFacetSet(FORD_ORD, 2010), new LongFacetSet(CHEVY_ORD, 2011)));
+    w.addDocument(doc);
+
+    // Ford-2011, Chevy-2010
+    doc = new Document();
+    doc.add(
+        FacetSetsField.create(
+            "field", new LongFacetSet(FORD_ORD, 2011), new LongFacetSet(CHEVY_ORD, 2010)));
+    w.addDocument(doc);
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    FacetsCollector fc = s.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    Facets facets =
+        new MatchingFacetSetsCounts(
+            "field",
+            fc,
+            FacetSetDecoder::decodeLongs,
+            new ExactFacetSetMatcher("Ford 2010", new LongFacetSet(FORD_ORD, 2010)),
+            new ExactFacetSetMatcher("Chevy 2011", new LongFacetSet(CHEVY_ORD, 2011)));
+
+    FacetResult result = facets.getAllChildren("field");
+
+    assertEquals("field", result.dim);
+    assertEquals(0, result.path.length);
+    assertEquals(1, result.value);
+    assertEquals(2, result.childCount);
+
+    assertEquals(new LabelAndValue("Ford 2010", 1), result.labelValues[0]);
+    assertEquals(new LabelAndValue("Chevy 2011", 1), result.labelValues[1]);
+
+    r.close();
+    d.close();
+  }
+
+  public void testLongFacetSetMatching() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+
+    List<LongFacetSet> allSets = new ArrayList<>();
+    for (int manufacturerOrd : MANUFACTURER_ORDS) {
+      for (int year : YEARS) {
+        allSets.add(new LongFacetSet(manufacturerOrd, year));
+      }
+    }
+
+    int numFord2010 = 0;
+    int numChevy2011 = 0;
+    int numMatchingDocs = 0;
+    for (int i = 0; i < 100; i++) {
+      Document doc = new Document();
+      int numSets = TestUtil.nextInt(random(), 1, 4);
+      Collections.shuffle(allSets, random());
+      LongFacetSet[] facetSets = allSets.subList(0, numSets).toArray(LongFacetSet[]::new);
+      boolean matchingDoc = false;
+      for (LongFacetSet facetSet : facetSets) {
+        if (FORD_ORD == facetSet.values[0] && facetSet.values[1] == 2010) {
+          ++numFord2010;
+          matchingDoc = true;
+        } else if (CHEVY_ORD == facetSet.values[0] && facetSet.values[1] == 2011) {
+          ++numChevy2011;
+          matchingDoc = true;
+        }
+      }
+      numMatchingDocs += matchingDoc ? 1 : 0;
+      doc.add(FacetSetsField.create("field", facetSets));
+      w.addDocument(doc);
+    }
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    FacetsCollector fc = s.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    Facets facets =
+        new MatchingFacetSetsCounts(
+            "field",
+            fc,
+            FacetSetDecoder::decodeLongs,
+            new ExactFacetSetMatcher("Ford 2010", new LongFacetSet(FORD_ORD, 2010)),
+            new ExactFacetSetMatcher("Chevy 2011", new LongFacetSet(CHEVY_ORD, 2011)));
+
+    FacetResult result = facets.getAllChildren("field");
+
+    assertEquals("field", result.dim);
+    assertEquals(0, result.path.length);
+    assertEquals(numMatchingDocs, result.value);
+    assertEquals(2, result.childCount);
+
+    assertEquals(new LabelAndValue("Ford 2010", numFord2010), result.labelValues[0]);
+    assertEquals(new LabelAndValue("Chevy 2011", numChevy2011), result.labelValues[1]);
+
+    r.close();
+    d.close();
+  }
+
+  public void testIntFacetSetMatching() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+
+    List<IntFacetSet> allSets = new ArrayList<>();
+    for (int manufacturerOrd : MANUFACTURER_ORDS) {
+      for (int year : YEARS) {
+        allSets.add(new IntFacetSet(manufacturerOrd, year));
+      }
+    }
+
+    int numFord2010 = 0;
+    int numChevy2011 = 0;
+    int numMatchingDocs = 0;
+    for (int i = 0; i < 100; i++) {
+      Document doc = new Document();
+      int numSets = TestUtil.nextInt(random(), 1, 4);
+      Collections.shuffle(allSets, random());
+      IntFacetSet[] facetSets = allSets.subList(0, numSets).toArray(IntFacetSet[]::new);
+      boolean matchingDoc = false;
+      for (IntFacetSet facetSet : facetSets) {
+        if (FORD_ORD == facetSet.values[0] && facetSet.values[1] == 2010) {
+          ++numFord2010;
+          matchingDoc = true;
+        } else if (CHEVY_ORD == facetSet.values[0] && facetSet.values[1] == 2011) {
+          ++numChevy2011;
+          matchingDoc = true;
+        }
+      }
+      numMatchingDocs += matchingDoc ? 1 : 0;
+      doc.add(FacetSetsField.create("field", facetSets));
+      w.addDocument(doc);
+    }
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    FacetsCollector fc = s.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    Facets facets =
+        new MatchingFacetSetsCounts(
+            "field",
+            fc,
+            FacetSetDecoder::decodeInts,
+            new ExactFacetSetMatcher("Ford 2010", new IntFacetSet(FORD_ORD, 2010)),
+            new ExactFacetSetMatcher("Chevy 2011", new IntFacetSet(CHEVY_ORD, 2011)));
+
+    FacetResult result = facets.getAllChildren("field");
+
+    assertEquals("field", result.dim);
+    assertEquals(0, result.path.length);
+    assertEquals(numMatchingDocs, result.value);
+    assertEquals(2, result.childCount);
+
+    assertEquals(new LabelAndValue("Ford 2010", numFord2010), result.labelValues[0]);
+    assertEquals(new LabelAndValue("Chevy 2011", numChevy2011), result.labelValues[1]);
+
+    r.close();
+    d.close();
+  }
+
+  public void testDoubleFacetSetMatching() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+
+    List<DoubleFacetSet> allSets = new ArrayList<>();
+    for (int manufacturerOrd : MANUFACTURER_ORDS) {
+      for (int year : YEARS) {
+        allSets.add(new DoubleFacetSet(manufacturerOrd, year + 0.5));
+      }
+    }
+
+    int numFord2010 = 0;
+    int numChevy2011 = 0;
+    int numMatchingDocs = 0;
+    for (int i = 0; i < 100; i++) {
+      Document doc = new Document();
+      int numSets = TestUtil.nextInt(random(), 1, 4);
+      Collections.shuffle(allSets, random());
+      DoubleFacetSet[] facetSets = allSets.subList(0, numSets).toArray(DoubleFacetSet[]::new);
+      boolean matchingDoc = false;
+      for (DoubleFacetSet facetSet : facetSets) {
+        if (FORD_ORD == facetSet.values[0] && facetSet.values[1] == 2010.5) {
+          ++numFord2010;
+          matchingDoc = true;
+        } else if (CHEVY_ORD == facetSet.values[0] && facetSet.values[1] == 2011.5) {
+          ++numChevy2011;
+          matchingDoc = true;
+        }
+      }
+      numMatchingDocs += matchingDoc ? 1 : 0;
+      doc.add(FacetSetsField.create("field", facetSets));
+      w.addDocument(doc);
+    }
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    FacetsCollector fc = s.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    Facets facets =
+        new MatchingFacetSetsCounts(
+            "field",
+            fc,
+            FacetSetDecoder::decodeLongs,
+            new ExactFacetSetMatcher("Ford 2010", new DoubleFacetSet(FORD_ORD, 2010.5)),
+            new ExactFacetSetMatcher("Chevy 2011", new DoubleFacetSet(CHEVY_ORD, 2011.5)));
+
+    FacetResult result = facets.getAllChildren("field");
+
+    assertEquals("field", result.dim);
+    assertEquals(0, result.path.length);
+    assertEquals(numMatchingDocs, result.value);
+    assertEquals(2, result.childCount);
+
+    assertEquals(new LabelAndValue("Ford 2010", numFord2010), result.labelValues[0]);
+    assertEquals(new LabelAndValue("Chevy 2011", numChevy2011), result.labelValues[1]);
+
+    r.close();
+    d.close();
+  }
+
+  public void testFloatFacetSetMatching() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+
+    List<FloatFacetSet> allSets = new ArrayList<>();
+    for (int manufacturerOrd : MANUFACTURER_ORDS) {
+      for (int year : YEARS) {
+        allSets.add(new FloatFacetSet(manufacturerOrd, year + 0.5f));
+      }
+    }
+
+    int numFord2010 = 0;
+    int numChevy2011 = 0;
+    int numMatchingDocs = 0;
+    for (int i = 0; i < 100; i++) {
+      Document doc = new Document();
+      int numSets = TestUtil.nextInt(random(), 1, 4);
+      Collections.shuffle(allSets, random());
+      FloatFacetSet[] facetSets = allSets.subList(0, numSets).toArray(FloatFacetSet[]::new);
+      boolean matchingDoc = false;
+      for (FloatFacetSet facetSet : facetSets) {
+        if (FORD_ORD == facetSet.values[0] && facetSet.values[1] == 2010.5f) {
+          ++numFord2010;
+          matchingDoc = true;
+        } else if (CHEVY_ORD == facetSet.values[0] && facetSet.values[1] == 2011.5f) {
+          ++numChevy2011;
+          matchingDoc = true;
+        }
+      }
+      numMatchingDocs += matchingDoc ? 1 : 0;
+      doc.add(FacetSetsField.create("field", facetSets));
+      w.addDocument(doc);
+    }
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    FacetsCollector fc = s.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    Facets facets =
+        new MatchingFacetSetsCounts(
+            "field",
+            fc,
+            FacetSetDecoder::decodeInts,
+            new ExactFacetSetMatcher("Ford 2010", new FloatFacetSet(FORD_ORD, 2010.5f)),
+            new ExactFacetSetMatcher("Chevy 2011", new FloatFacetSet(CHEVY_ORD, 2011.5f)));
+
+    FacetResult result = facets.getAllChildren("field");
+
+    assertEquals("field", result.dim);
+    assertEquals(0, result.path.length);
+    assertEquals(numMatchingDocs, result.value);
+    assertEquals(2, result.childCount);
+
+    assertEquals(new LabelAndValue("Ford 2010", numFord2010), result.labelValues[0]);
+    assertEquals(new LabelAndValue("Chevy 2011", numChevy2011), result.labelValues[1]);
+
+    r.close();
+    d.close();
+  }
+}
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/facetset/TestMatchingFacetSetsCounts.java b/lucene/facet/src/test/org/apache/lucene/facet/facetset/TestMatchingFacetSetsCounts.java
new file mode 100644
index 00000000000..51466137f9e
--- /dev/null
+++ b/lucene/facet/src/test/org/apache/lucene/facet/facetset/TestMatchingFacetSetsCounts.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.facetset;
+
+import java.io.IOException;
+import org.apache.lucene.document.Document;
+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 TestMatchingFacetSetsCounts extends FacetTestCase {
+
+  public void testInvalidTopN() throws IOException {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+
+    Document doc = new Document();
+    doc.add(FacetSetsField.create("field", new LongFacetSet(123, 456)));
+    w.addDocument(doc);
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    FacetsCollector fc = s.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    Facets facets =
+        new MatchingFacetSetsCounts(
+            "field",
+            fc,
+            FacetSetDecoder::decodeLongs,
+            new ExactFacetSetMatcher("Test", new LongFacetSet(123, 456)));
+
+    expectThrows(IllegalArgumentException.class, () -> facets.getTopChildren(0, "field"));
+
+    r.close();
+    d.close();
+  }
+
+  public void testInconsistentNumOfIndexedDimensions() throws IOException {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+
+    Document doc = new Document();
+    doc.add(FacetSetsField.create("field", new LongFacetSet(123, 456)));
+    w.addDocument(doc);
+
+    doc = new Document();
+    doc.add(FacetSetsField.create("field", new LongFacetSet(123)));
+    w.addDocument(doc);
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    FacetsCollector fc = s.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    expectThrows(
+        AssertionError.class,
+        () ->
+            new MatchingFacetSetsCounts(
+                "field",
+                fc,
+                FacetSetDecoder::decodeLongs,
+                new ExactFacetSetMatcher("Test", new LongFacetSet(1))));
+
+    r.close();
+    d.close();
+  }
+}
diff --git a/lucene/facet/src/test/org/apache/lucene/facet/facetset/TestRangeFacetSetMatcher.java b/lucene/facet/src/test/org/apache/lucene/facet/facetset/TestRangeFacetSetMatcher.java
new file mode 100644
index 00000000000..ec055d900df
--- /dev/null
+++ b/lucene/facet/src/test/org/apache/lucene/facet/facetset/TestRangeFacetSetMatcher.java
@@ -0,0 +1,418 @@
+/*
+ * 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.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.facet.*;
+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;
+import org.apache.lucene.tests.util.TestUtil;
+
+public class TestRangeFacetSetMatcher extends FacetTestCase {
+
+  private static final int FORD_ORD = 100;
+  private static final int TOYOTA_ORD = 101;
+  private static final int CHEVY_ORD = 102;
+  private static final int NISSAN_ORD = 103;
+  private static final int[] MANUFACTURER_ORDS = {FORD_ORD, TOYOTA_ORD, CHEVY_ORD, NISSAN_ORD};
+  private static final int[] YEARS = {2010, 2011, 2012, 2013, 2014};
+
+  public void testLongRangeFacetSetMatching() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+
+    List<LongFacetSet> allSets = new ArrayList<>();
+    for (int manufacturerOrd : MANUFACTURER_ORDS) {
+      for (int year : YEARS) {
+        allSets.add(new LongFacetSet(manufacturerOrd, year));
+      }
+    }
+
+    int numFord2011_2013 = 0;
+    int numFord2010_2014 = 0;
+    int numFord2011_2014 = 0;
+    int numFord2010_2013 = 0;
+    int numMatchingDocs = 0;
+    for (int i = 0; i < 100; i++) {
+      Document doc = new Document();
+      int numSets = TestUtil.nextInt(random(), 1, 4);
+      Collections.shuffle(allSets, random());
+      LongFacetSet[] facetSets = allSets.subList(0, numSets).toArray(LongFacetSet[]::new);
+      boolean matchingDoc = false;
+      for (LongFacetSet facetSet : facetSets) {
+        if (FORD_ORD != facetSet.values[0]) {
+          continue;
+        }
+        long year = facetSet.values[1];
+        if (year > 2010 && year < 2014) {
+          ++numFord2010_2013;
+          ++numFord2010_2014;
+          ++numFord2011_2013;
+          ++numFord2011_2014;
+          matchingDoc = true;
+        } else if (year == 2014) {
+          ++numFord2010_2014;
+          ++numFord2011_2014;
+          matchingDoc = true;
+        } else if (year == 2010) {
+          ++numFord2010_2014;
+          ++numFord2010_2013;
+          matchingDoc = true;
+        }
+      }
+      numMatchingDocs += matchingDoc ? 1 : 0;
+      doc.add(FacetSetsField.create("field", facetSets));
+      w.addDocument(doc);
+    }
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    FacetsCollector fc = s.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    Facets facets =
+        new MatchingFacetSetsCounts(
+            "field",
+            fc,
+            FacetSetDecoder::decodeLongs,
+            new RangeFacetSetMatcher(
+                "Ford [2010-2014]", singleLong(FORD_ORD), longRange(2010, true, 2014, true)),
+            new RangeFacetSetMatcher(
+                "Ford (2010-2014]", singleLong(FORD_ORD), longRange(2010, false, 2014, true)),
+            new RangeFacetSetMatcher(
+                "Ford [2010-2014)", singleLong(FORD_ORD), longRange(2010, true, 2014, false)),
+            new RangeFacetSetMatcher(
+                "Ford (2010-2014)", singleLong(FORD_ORD), longRange(2010, false, 2014, false)));
+
+    FacetResult result = facets.getAllChildren("field");
+
+    assertEquals("field", result.dim);
+    assertEquals(0, result.path.length);
+    assertEquals(numMatchingDocs, result.value);
+    assertEquals(4, result.childCount);
+
+    assertEquals(new LabelAndValue("Ford [2010-2014]", numFord2010_2014), result.labelValues[0]);
+    assertEquals(new LabelAndValue("Ford (2010-2014]", numFord2011_2014), result.labelValues[1]);
+    assertEquals(new LabelAndValue("Ford [2010-2014)", numFord2010_2013), result.labelValues[2]);
+    assertEquals(new LabelAndValue("Ford (2010-2014)", numFord2011_2013), result.labelValues[3]);
+
+    r.close();
+    d.close();
+  }
+
+  public void testIntRangeFacetSetMatching() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+
+    List<IntFacetSet> allSets = new ArrayList<>();
+    for (int manufacturerOrd : MANUFACTURER_ORDS) {
+      for (int year : YEARS) {
+        allSets.add(new IntFacetSet(manufacturerOrd, year));
+      }
+    }
+
+    int numFord2011_2013 = 0;
+    int numFord2010_2014 = 0;
+    int numFord2011_2014 = 0;
+    int numFord2010_2013 = 0;
+    int numMatchingDocs = 0;
+    for (int i = 0; i < 100; i++) {
+      Document doc = new Document();
+      int numSets = TestUtil.nextInt(random(), 1, 4);
+      Collections.shuffle(allSets, random());
+      IntFacetSet[] facetSets = allSets.subList(0, numSets).toArray(IntFacetSet[]::new);
+      boolean matchingDoc = false;
+      for (IntFacetSet facetSet : facetSets) {
+        if (FORD_ORD != facetSet.values[0]) {
+          continue;
+        }
+        int year = facetSet.values[1];
+        if (year > 2010 && year < 2014) {
+          ++numFord2010_2013;
+          ++numFord2010_2014;
+          ++numFord2011_2013;
+          ++numFord2011_2014;
+          matchingDoc = true;
+        } else if (year == 2014) {
+          ++numFord2010_2014;
+          ++numFord2011_2014;
+          matchingDoc = true;
+        } else if (year == 2010) {
+          ++numFord2010_2014;
+          ++numFord2010_2013;
+          matchingDoc = true;
+        }
+      }
+      numMatchingDocs += matchingDoc ? 1 : 0;
+      doc.add(FacetSetsField.create("field", facetSets));
+      w.addDocument(doc);
+    }
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    FacetsCollector fc = s.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    Facets facets =
+        new MatchingFacetSetsCounts(
+            "field",
+            fc,
+            FacetSetDecoder::decodeInts,
+            new RangeFacetSetMatcher(
+                "Ford [2010-2014]", singleLong(FORD_ORD), longRange(2010, true, 2014, true)),
+            new RangeFacetSetMatcher(
+                "Ford (2010-2014]", singleLong(FORD_ORD), longRange(2010, false, 2014, true)),
+            new RangeFacetSetMatcher(
+                "Ford [2010-2014)", singleLong(FORD_ORD), longRange(2010, true, 2014, false)),
+            new RangeFacetSetMatcher(
+                "Ford (2010-2014)", singleLong(FORD_ORD), longRange(2010, false, 2014, false)));
+
+    FacetResult result = facets.getAllChildren("field");
+
+    assertEquals("field", result.dim);
+    assertEquals(0, result.path.length);
+    assertEquals(numMatchingDocs, result.value);
+    assertEquals(4, result.childCount);
+
+    assertEquals(new LabelAndValue("Ford [2010-2014]", numFord2010_2014), result.labelValues[0]);
+    assertEquals(new LabelAndValue("Ford (2010-2014]", numFord2011_2014), result.labelValues[1]);
+    assertEquals(new LabelAndValue("Ford [2010-2014)", numFord2010_2013), result.labelValues[2]);
+    assertEquals(new LabelAndValue("Ford (2010-2014)", numFord2011_2013), result.labelValues[3]);
+
+    r.close();
+    d.close();
+  }
+
+  public void testDoubleRangeFacetSetMatching() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+
+    List<DoubleFacetSet> allSets = new ArrayList<>();
+    for (int manufacturerOrd : MANUFACTURER_ORDS) {
+      for (int year : YEARS) {
+        allSets.add(new DoubleFacetSet(manufacturerOrd, year + 0.5));
+      }
+    }
+
+    int numFord2011_2014 = 0;
+    int numFord2010_2015 = 0;
+    int numFord2011_2015 = 0;
+    int numFord2010_2014 = 0;
+    int numMatchingDocs = 0;
+    for (int i = 0; i < 100; i++) {
+      Document doc = new Document();
+      int numSets = TestUtil.nextInt(random(), 1, 4);
+      Collections.shuffle(allSets, random());
+      DoubleFacetSet[] facetSets = allSets.subList(0, numSets).toArray(DoubleFacetSet[]::new);
+      boolean matchingDoc = false;
+      for (DoubleFacetSet facetSet : facetSets) {
+        if (Double.compare(FORD_ORD, facetSet.values[0]) != 0) {
+          continue;
+        }
+        double year = facetSet.values[1];
+        if (year > 2010.5 && year < 2014.5) {
+          ++numFord2010_2014;
+          ++numFord2010_2015;
+          ++numFord2011_2014;
+          ++numFord2011_2015;
+          matchingDoc = true;
+        } else if (year == 2014.5) {
+          ++numFord2010_2015;
+          ++numFord2011_2015;
+          matchingDoc = true;
+        } else if (year == 2010.5) {
+          ++numFord2010_2015;
+          ++numFord2010_2014;
+          matchingDoc = true;
+        }
+      }
+      numMatchingDocs += matchingDoc ? 1 : 0;
+      doc.add(FacetSetsField.create("field", facetSets));
+      w.addDocument(doc);
+    }
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    FacetsCollector fc = s.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    Facets facets =
+        new MatchingFacetSetsCounts(
+            "field",
+            fc,
+            FacetSetDecoder::decodeLongs,
+            new RangeFacetSetMatcher(
+                "Ford [2010-2015]",
+                singleDouble(FORD_ORD),
+                doubleRange(2010.0, true, 2015.0, true)),
+            new RangeFacetSetMatcher(
+                "Ford (2011-2015]",
+                singleDouble(FORD_ORD),
+                doubleRange(2011.0, false, 2015.0, true)),
+            new RangeFacetSetMatcher(
+                "Ford [2010-2014)",
+                singleDouble(FORD_ORD),
+                doubleRange(2010.0, true, 2014.0, false)),
+            new RangeFacetSetMatcher(
+                "Ford (2011-2014)",
+                singleDouble(FORD_ORD),
+                doubleRange(2011.0, false, 2014.0, false)));
+
+    FacetResult result = facets.getAllChildren("field");
+
+    assertEquals("field", result.dim);
+    assertEquals(0, result.path.length);
+    assertEquals(numMatchingDocs, result.value);
+    assertEquals(4, result.childCount);
+
+    assertEquals(new LabelAndValue("Ford [2010-2015]", numFord2010_2015), result.labelValues[0]);
+    assertEquals(new LabelAndValue("Ford (2011-2015]", numFord2011_2015), result.labelValues[1]);
+    assertEquals(new LabelAndValue("Ford [2010-2014)", numFord2010_2014), result.labelValues[2]);
+    assertEquals(new LabelAndValue("Ford (2011-2014)", numFord2011_2014), result.labelValues[3]);
+
+    r.close();
+    d.close();
+  }
+
+  public void testFloatRangeFacetSetMatching() throws Exception {
+    Directory d = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), d);
+
+    List<FloatFacetSet> allSets = new ArrayList<>();
+    for (int manufacturerOrd : MANUFACTURER_ORDS) {
+      for (int year : YEARS) {
+        allSets.add(new FloatFacetSet(manufacturerOrd, year + 0.5f));
+      }
+    }
+
+    int numFord2011_2014 = 0;
+    int numFord2010_2015 = 0;
+    int numFord2011_2015 = 0;
+    int numFord2010_2014 = 0;
+    int numMatchingDocs = 0;
+    for (int i = 0; i < 100; i++) {
+      Document doc = new Document();
+      int numSets = TestUtil.nextInt(random(), 1, 4);
+      Collections.shuffle(allSets, random());
+      FloatFacetSet[] facetSets = allSets.subList(0, numSets).toArray(FloatFacetSet[]::new);
+      boolean matchingDoc = false;
+      for (FloatFacetSet facetSet : facetSets) {
+        if (Double.compare(FORD_ORD, facetSet.values[0]) != 0) {
+          continue;
+        }
+        double year = facetSet.values[1];
+        if (year > 2010.5f && year < 2014.5f) {
+          ++numFord2010_2014;
+          ++numFord2010_2015;
+          ++numFord2011_2014;
+          ++numFord2011_2015;
+          matchingDoc = true;
+        } else if (year == 2014.5f) {
+          ++numFord2010_2015;
+          ++numFord2011_2015;
+          matchingDoc = true;
+        } else if (year == 2010.5f) {
+          ++numFord2010_2015;
+          ++numFord2010_2014;
+          matchingDoc = true;
+        }
+      }
+      numMatchingDocs += matchingDoc ? 1 : 0;
+      doc.add(FacetSetsField.create("field", facetSets));
+      w.addDocument(doc);
+    }
+
+    IndexReader r = w.getReader();
+    w.close();
+
+    IndexSearcher s = newSearcher(r);
+    FacetsCollector fc = s.search(new MatchAllDocsQuery(), new FacetsCollectorManager());
+
+    Facets facets =
+        new MatchingFacetSetsCounts(
+            "field",
+            fc,
+            FacetSetDecoder::decodeInts,
+            new RangeFacetSetMatcher(
+                "Ford [2010-2015]",
+                singleFloat(FORD_ORD),
+                floatRange(2010.0f, true, 2015.0f, true)),
+            new RangeFacetSetMatcher(
+                "Ford (2010-2015]",
+                singleFloat(FORD_ORD),
+                floatRange(2010.5f, false, 2015.0f, true)),
+            new RangeFacetSetMatcher(
+                "Ford [2010-2014)",
+                singleFloat(FORD_ORD),
+                floatRange(2010.0f, true, 2014.0f, false)),
+            new RangeFacetSetMatcher(
+                "Ford (2011-2014)",
+                singleFloat(FORD_ORD),
+                floatRange(2011.0f, false, 2014.0f, false)));
+
+    FacetResult result = facets.getAllChildren("field");
+
+    assertEquals("field", result.dim);
+    assertEquals(0, result.path.length);
+    assertEquals(numMatchingDocs, result.value);
+    assertEquals(4, result.childCount);
+
+    assertEquals(new LabelAndValue("Ford [2010-2015]", numFord2010_2015), result.labelValues[0]);
+    assertEquals(new LabelAndValue("Ford (2010-2015]", numFord2011_2015), result.labelValues[1]);
+    assertEquals(new LabelAndValue("Ford [2010-2014)", numFord2010_2014), result.labelValues[2]);
+    assertEquals(new LabelAndValue("Ford (2011-2014)", numFord2011_2014), result.labelValues[3]);
+
+    r.close();
+    d.close();
+  }
+
+  private static DimRange singleLong(long value) {
+    return DimRange.fromLongs(value, true, value, true);
+  }
+
+  private static DimRange longRange(
+      long min, boolean minExclusive, long max, boolean maxExclusive) {
+    return DimRange.fromLongs(min, minExclusive, max, maxExclusive);
+  }
+
+  private static DimRange singleDouble(double value) {
+    return DimRange.fromDoubles(value, true, value, true);
+  }
+
+  private static DimRange doubleRange(
+      double min, boolean minExclusive, double max, boolean maxExclusive) {
+    return DimRange.fromDoubles(min, minExclusive, max, maxExclusive);
+  }
+
+  private static DimRange singleFloat(float value) {
+    return DimRange.fromFloats(value, true, value, true);
+  }
+
+  private static DimRange floatRange(
+      float min, boolean minExclusive, float max, boolean maxExclusive) {
+    return DimRange.fromFloats(min, minExclusive, max, maxExclusive);
+  }
+}