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


##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/Reducer.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.sandbox.facet.recorders;
+
+/** Reducer for numeric values. */
+public interface Reducer {
+
+  /** Int values reducer. */
+  int reduce(int a, int b);
+
+  /** Long values reducer. */
+  long reduce(long a, long b);
+
+  /** Float values reducer. */
+  float reduce(float a, float b);
+
+  /** Double values reducer. */
+  double reduce(double a, double b);
+  ;

Review Comment:
   nit: stray semi-colon (tag: @mikemccand  ;) )



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/ordinals/CandidateSetOrdinalIterator.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.sandbox.facet.ordinals;
+
+import java.io.IOException;
+import org.apache.lucene.internal.hppc.IntHashSet;
+
+/**
+ * {@link OrdinalIterator} that filters out ordinals from delegate if they are 
not in the candidate
+ * set.
+ *
+ * <p>Can be handy to get results only for specific facets.
+ */
+public class CandidateSetOrdinalIterator implements OrdinalIterator {

Review Comment:
   Maybe final?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/LongRangeFacetCutter.java:
##########
@@ -0,0 +1,413 @@
+/*
+ * 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.sandbox.facet.cutters.ranges;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+import org.apache.lucene.facet.MultiLongValues;
+import org.apache.lucene.facet.MultiLongValuesSource;
+import org.apache.lucene.facet.range.LongRange;
+import org.apache.lucene.sandbox.facet.cutters.FacetCutter;
+import org.apache.lucene.sandbox.facet.cutters.LeafFacetCutter;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
+
+/** {@link RangeFacetCutter} for ranges of long values. It's based on 
LongRangeCounter class. */
+public abstract class LongRangeFacetCutter extends RangeFacetCutter {
+
+  MultiLongValuesSource valuesSource;
+  LongValuesSource singleValues; // TODO: refactor - weird that we have both 
multi and single here.
+  LongRangeAndPos[] sortedRanges;
+
+  int requestedRangeCount;
+
+  List<InclusiveRange> elementaryIntervals;
+
+  /** elementary interval boundaries used for efficient counting (bsearch to 
find interval) */
+  long[] boundaries;
+
+  int[] pos;
+
+  // Default interval position, when elementary interval is mapped to this 
interval
+  // it is skipped.
+  static final int SKIP_INTERVAL_POSITION = -1;
+
+  /** Create {@link FacetCutter} for provided value source and long ranges. */
+  public static LongRangeFacetCutter create(
+      String field,
+      MultiLongValuesSource longValuesSource,
+      LongValuesSource singleLongValuesSource,
+      LongRange[] longRanges) {
+    if (areOverlappingRanges(longRanges)) {
+      return new OverlappingLongRangeFacetCutter(
+          field, longValuesSource, singleLongValuesSource, longRanges);
+    }
+    return new NonOverlappingLongRangeFacetCutter(
+        field, longValuesSource, singleLongValuesSource, longRanges);
+  }
+
+  public static LongRangeFacetCutter create(
+      String field, MultiLongValuesSource longValuesSource, LongRange[] 
longRanges) {
+    return create(field, longValuesSource, null, longRanges);
+  }
+
+  // caller handles conversion of Doubles and DoubleRange to Long and LongRange
+  // ranges need not be sorted
+  LongRangeFacetCutter(
+      String field,
+      MultiLongValuesSource longValuesSource,
+      LongValuesSource singleLongValuesSource,
+      LongRange[] longRanges) {
+    super(field);
+    valuesSource = longValuesSource;
+    if (singleLongValuesSource != null) {
+      singleValues = singleLongValuesSource;
+    } else {
+      singleValues = MultiLongValuesSource.unwrapSingleton(valuesSource);
+    }
+
+    sortedRanges = new LongRangeAndPos[longRanges.length];
+    requestedRangeCount = longRanges.length;
+
+    for (int i = 0; i < longRanges.length; i++) {
+      sortedRanges[i] = new LongRangeAndPos(longRanges[i], i);
+    }
+
+    Arrays.sort(this.sortedRanges, Comparator.comparingLong(r -> r.range.min));
+    elementaryIntervals = buildElementaryIntervals();
+
+    // Keep track of elementary interval boundary ends (for binary search) 
along with the requested
+    // range they map back to (and -1 when they map to a "gap" range in case 
of ExclusiveRanges):
+    boundaries = new long[elementaryIntervals.size()];
+    pos = new int[elementaryIntervals.size()];
+    Arrays.fill(pos, SKIP_INTERVAL_POSITION);
+    int currRange = 0;
+    for (int i = 0; i < boundaries.length; i++) {
+      boundaries[i] = elementaryIntervals.get(i).end;
+      if (currRange < sortedRanges.length) {
+        LongRangeAndPos curr = sortedRanges[currRange];
+        if (boundaries[i] == curr.range.max) {
+          pos[i] = curr.pos;
+          currRange++;
+        }
+      }
+    }
+  }
+
+  abstract List<InclusiveRange> buildElementaryIntervals();
+
+  private static boolean areOverlappingRanges(LongRange[] ranges) {
+    if (ranges.length == 0) {
+      return false;
+    }
+
+    // Copy before sorting so we don't mess with the caller's original ranges:
+    // TODO: We're going to do this again in the constructor. Can't we come up 
with a clever way to
+    // avoid doing it twice?
+    LongRange[] sortedRanges = new LongRange[ranges.length];
+    System.arraycopy(ranges, 0, sortedRanges, 0, ranges.length);
+    Arrays.sort(sortedRanges, Comparator.comparingLong(r -> r.min));
+
+    long previousMax = sortedRanges[0].max;
+    for (int i = 1; i < sortedRanges.length; i++) {
+      // Ranges overlap if the next min is <= the previous max (note that 
LongRange models
+      // closed ranges, so equal limit points are considered overlapping):
+      if (sortedRanges[i].min <= previousMax) {
+        return true;
+      }
+      previousMax = sortedRanges[i].max;
+    }
+
+    return false;
+  }
+
+  abstract static class LongRangeMultivaluedLeafFacetCutter implements 
LeafFacetCutter {
+    final MultiLongValues multiLongValues;
+    final long[] boundaries;
+    final int[] pos;
+
+    final int requestedRangeCount;
+
+    // int currentDoc = -1;
+
+    final IntervalTracker elementaryIntervalTracker;
+
+    // TODO: we need it only for overlapping ranges, should not handle it in 
advanceExact for
+    // exclusive ranges.
+    IntervalTracker requestedIntervalTracker;
+
+    LongRangeMultivaluedLeafFacetCutter(
+        MultiLongValues longValues, long[] boundaries, int[] pos, int 
requestedRangeCount) {
+      this.multiLongValues = longValues;
+      this.boundaries = boundaries;
+      this.pos = pos;
+      this.requestedRangeCount = requestedRangeCount;
+      elementaryIntervalTracker = new 
IntervalTracker.MultiIntervalTracker(boundaries.length);
+    }
+
+    @Override
+    public boolean advanceExact(int doc) throws IOException {
+      if (multiLongValues.advanceExact(doc) == false) {
+        return false;
+      }
+
+      elementaryIntervalTracker.clear();
+
+      if (requestedIntervalTracker != null) {
+        requestedIntervalTracker.clear();
+      }
+
+      long numValues = multiLongValues.getValueCount();
+
+      int lastIntervalSeen = -1;
+
+      for (int i = 0; i < numValues; i++) {
+        lastIntervalSeen = processValue(multiLongValues.nextValue(), 
lastIntervalSeen);
+        if (lastIntervalSeen >= 0 && lastIntervalSeen < boundaries.length) {
+          elementaryIntervalTracker.set(lastIntervalSeen);
+        }
+        if (lastIntervalSeen == boundaries.length - 1) {
+          // we've already reached the end of all possible intervals for this 
doc
+          break;
+        }
+      }
+      maybeRollUp(requestedIntervalTracker);
+
+      elementaryIntervalTracker.freeze();
+
+      if (requestedIntervalTracker != null) {
+        requestedIntervalTracker.freeze();
+      }
+
+      return true;
+    }
+
+    // Returns the value of the interval v belongs or lastIntervalSeen
+    // if no processing is done, it returns the lastIntervalSeen
+    private int processValue(long v, int lastIntervalSeen) {
+      int lo = 0, hi = boundaries.length - 1;
+      ;
+
+      if (lastIntervalSeen != -1) {
+        // this is the multivalued doc case, we need to set lo correctly
+        if (v <= boundaries[lastIntervalSeen]) {
+          // we've already counted something for this interval and doc
+          // we don't need to process v
+          return lastIntervalSeen;
+        }
+
+        lo = lastIntervalSeen + 1;
+        if (lo == boundaries.length) {
+          // we've already counted the last elementary interval. If so, 
there's nothing
+          // else to count for this doc
+          // TODO: does it make sense to return something else?
+          return lastIntervalSeen;
+        }
+      }
+      int lowerBound = lo;
+
+      while (true) {
+        int mid = (lo + hi) >>> 1;
+        if (v <= boundaries[mid]) {
+          if (mid == lowerBound) {
+            return mid;
+          } else {
+            hi = mid - 1;
+          }
+        } else if (v > boundaries[mid + 1]) {
+          lo = mid + 1;
+        } else {
+          return mid + 1;
+        }
+      }
+    }
+
+    void maybeRollUp(IntervalTracker rollUpInto) {}
+  }
+
+  abstract static class LongRangeSingleValuedLeafFacetCutter implements 
LeafFacetCutter {
+    final LongValues longValues;
+    final long[] boundaries;
+    final int[] pos;
+
+    final int requestedRangeCount;
+    final IntervalTracker elementaryIntervalTracker;
+
+    IntervalTracker requestedIntervalTracker;
+
+    LongRangeSingleValuedLeafFacetCutter(
+        LongValues longValues, long[] boundaries, int[] pos, int 
requestedRangeCount) {
+      this.longValues = longValues;
+      this.boundaries = boundaries;
+      this.pos = pos;
+      this.requestedRangeCount = requestedRangeCount;
+      elementaryIntervalTracker = new IntervalTracker.SingleIntervalTracker();
+    }
+
+    @Override
+    public boolean advanceExact(int doc) throws IOException {
+      if (longValues.advanceExact(doc) == false) {
+        return false;
+      }
+
+      elementaryIntervalTracker.clear();
+
+      if (requestedIntervalTracker != null) {
+        requestedIntervalTracker.clear();
+      }
+
+      int lastIntervalSeen = -1;
+
+      lastIntervalSeen = processValue(longValues.longValue(), 
lastIntervalSeen);
+      elementaryIntervalTracker.set(lastIntervalSeen);
+
+      maybeRollUp(requestedIntervalTracker);
+
+      // if (elementaryIntervalTracker != null) {
+      elementaryIntervalTracker.freeze();
+      // }
+      if (requestedIntervalTracker != null) {
+        requestedIntervalTracker.freeze();
+      }
+
+      return true;
+    }
+
+    // Returns the value of the interval v belongs or lastIntervalSeen
+    // if no processing is done, it returns the lastIntervalSeen
+    // TODO: dedup with multi valued?
+    private int processValue(long v, int lastIntervalSeen) {
+      int lo = 0, hi = boundaries.length - 1;
+      ;
+
+      if (lastIntervalSeen != -1) {
+        // this is the multivalued doc case, we need to set lo correctly
+        if (v <= boundaries[lastIntervalSeen]) {
+          // we've already counted something for this interval and doc
+          // we don't need to process v
+          return lastIntervalSeen;
+        }
+
+        lo = lastIntervalSeen + 1;
+        if (lo == boundaries.length) {
+          // we've already counted the last elementary interval. If so, 
there's nothing
+          // else to count for this doc
+          // TODO: does it make sense to return something else?
+          return lastIntervalSeen;
+        }
+      }
+      int lowerBound = lo;
+
+      while (true) {
+        int mid = (lo + hi) >>> 1;
+        if (v <= boundaries[mid]) {
+          if (mid == lowerBound) {
+            return mid;
+          } else {
+            hi = mid - 1;
+          }
+        } else if (v > boundaries[mid + 1]) {
+          lo = mid + 1;
+        } else {
+          return mid + 1;
+        }
+      }
+    }
+
+    void maybeRollUp(IntervalTracker rollUpInto) {}
+  }
+
+  static final class LongRangeAndPos {

Review Comment:
   Let's use `record` here?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/ordinals/OrdinalGetter.java:
##########
@@ -0,0 +1,24 @@
+/*
+ * 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.sandbox.facet.ordinals;
+
+/** Interface to return an ordinal. */
+public interface OrdinalGetter {

Review Comment:
   What if we just use Java's `IntSupplier` instead of creating our own 
interface?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/ordinals/CandidateSetOrdinalIterator.java:
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.sandbox.facet.ordinals;
+
+import java.io.IOException;
+import org.apache.lucene.internal.hppc.IntHashSet;
+
+/**
+ * {@link OrdinalIterator} that filters out ordinals from delegate if they are 
not in the candidate

Review Comment:
   What's the use-case for this? I'm a little worried this will promote 
inefficient faceting patterns. If the caller has a set of specific ordinals 
they're interested in getting counts/aggregations for, wouldn't they be better 
off just directly accessing the recorder(s) with those ordinals? Why would they 
want to filter an iterator of all ordinals the recorder saw? Maybe I'm missing 
the use-case though?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/CountFacetRecorder.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.sandbox.facet.recorders;
+
+import static 
org.apache.lucene.sandbox.facet.ordinals.OrdinalIterator.NO_MORE_ORDS;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.internal.hppc.IntCursor;
+import org.apache.lucene.internal.hppc.IntIntHashMap;
+import org.apache.lucene.sandbox.facet.cutters.LeafFacetCutter;
+import org.apache.lucene.sandbox.facet.misc.FacetRollup;
+import org.apache.lucene.sandbox.facet.ordinals.OrdinalIterator;
+
+/**
+ * {@link FacetRecorder} to count facets.
+ *
+ * <p>TODO: add an option to keep counts in an array, to improve performance 
for facets with small
+ * number of ordinals e.g. range facets. Options: - {@link LeafFacetCutter} 
can inform {@link
+ * LeafFacetRecorder} about expected number of facet ordinals ({@link
+ * org.apache.lucene.sandbox.facet.FacetFieldCollector} can orchestrate that). 
If expeted facet ord
+ * number is below some threshold - use array instead of a map? - first 100/1k 
counts in array, the
+ * rest - in a map; the limit can also be provided in a constructor? It is 
similar to what
+ * LongValuesFacetCounts does today.
+ *
+ * <p>TODO: We can also consider collecting 2 (3, 4, ..., can be parametrizes) 
slices to a single
+ * sync map which can reduce thread contention compared to single sync map for 
all slices; at the
+ * same time there will be less work for reduce method. So far reduce wasn't a 
bottleneck for us,
+ * but it is definitely not free.
+ *
+ * <p>TODO: If we come back to some for of synchronized count maps, we should 
be more careful what
+ * we acquire locks for - we used to lock addTo method itself, but it could be 
faster if we only
+ * synchronized after computing the key's hash; or we can lock the entire map 
only if we need to
+ * insert key, and lock single key otherwise?
+ */
+public class CountFacetRecorder implements FacetRecorder {

Review Comment:
   nit: maybe final?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/LongAggregationsFacetRecorder.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.sandbox.facet.recorders;
+
+import static 
org.apache.lucene.sandbox.facet.ordinals.OrdinalIterator.NO_MORE_ORDS;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.internal.hppc.IntCursor;
+import org.apache.lucene.internal.hppc.IntObjectHashMap;
+import org.apache.lucene.sandbox.facet.misc.FacetRollup;
+import org.apache.lucene.sandbox.facet.ordinals.OrdinalIterator;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
+
+/**
+ * {@link FacetRecorder} that computes multiple long aggregations per facet.
+ *
+ * <p>TODO: [premature optimization idea] if instead of one array we keep 
aggregations in two
+ * LongVector (one for MAX aggregation and one for SUM) we can benefit from 
SIMD?
+ */
+public class LongAggregationsFacetRecorder implements FacetRecorder {

Review Comment:
   nit: should we make this class final?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/LongAggregationsFacetRecorder.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.sandbox.facet.recorders;
+
+import static 
org.apache.lucene.sandbox.facet.ordinals.OrdinalIterator.NO_MORE_ORDS;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.internal.hppc.IntCursor;
+import org.apache.lucene.internal.hppc.IntObjectHashMap;
+import org.apache.lucene.sandbox.facet.misc.FacetRollup;
+import org.apache.lucene.sandbox.facet.ordinals.OrdinalIterator;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
+
+/**
+ * {@link FacetRecorder} that computes multiple long aggregations per facet.
+ *
+ * <p>TODO: [premature optimization idea] if instead of one array we keep 
aggregations in two
+ * LongVector (one for MAX aggregation and one for SUM) we can benefit from 
SIMD?
+ */
+public class LongAggregationsFacetRecorder implements FacetRecorder {
+
+  private IntObjectHashMap<long[]> values;
+  private List<IntObjectHashMap<long[]>> leafValues;
+
+  private final LongValuesSource[] longValuesSources;
+  private final Reducer[] reducers;
+
+  /** Constructor. */
+  public LongAggregationsFacetRecorder(LongValuesSource[] longValuesSources, 
Reducer[] reducers) {
+    assert longValuesSources.length == reducers.length;
+    this.longValuesSources = longValuesSources;
+    this.reducers = reducers;
+    values = new IntObjectHashMap<>();

Review Comment:
   Do we need to initialize this here?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/CountFacetRecorder.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.sandbox.facet.recorders;
+
+import static 
org.apache.lucene.sandbox.facet.ordinals.OrdinalIterator.NO_MORE_ORDS;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.internal.hppc.IntCursor;
+import org.apache.lucene.internal.hppc.IntIntHashMap;
+import org.apache.lucene.sandbox.facet.cutters.LeafFacetCutter;
+import org.apache.lucene.sandbox.facet.misc.FacetRollup;
+import org.apache.lucene.sandbox.facet.ordinals.OrdinalIterator;
+
+/**
+ * {@link FacetRecorder} to count facets.
+ *
+ * <p>TODO: add an option to keep counts in an array, to improve performance 
for facets with small
+ * number of ordinals e.g. range facets. Options: - {@link LeafFacetCutter} 
can inform {@link
+ * LeafFacetRecorder} about expected number of facet ordinals ({@link
+ * org.apache.lucene.sandbox.facet.FacetFieldCollector} can orchestrate that). 
If expeted facet ord
+ * number is below some threshold - use array instead of a map? - first 100/1k 
counts in array, the
+ * rest - in a map; the limit can also be provided in a constructor? It is 
similar to what
+ * LongValuesFacetCounts does today.
+ *
+ * <p>TODO: We can also consider collecting 2 (3, 4, ..., can be parametrizes) 
slices to a single
+ * sync map which can reduce thread contention compared to single sync map for 
all slices; at the
+ * same time there will be less work for reduce method. So far reduce wasn't a 
bottleneck for us,
+ * but it is definitely not free.
+ *
+ * <p>TODO: If we come back to some for of synchronized count maps, we should 
be more careful what
+ * we acquire locks for - we used to lock addTo method itself, but it could be 
faster if we only
+ * synchronized after computing the key's hash; or we can lock the entire map 
only if we need to
+ * insert key, and lock single key otherwise?
+ */
+public class CountFacetRecorder implements FacetRecorder {
+  IntIntHashMap values;
+  List<IntIntHashMap> perLeafValues;
+
+  /** Create. */
+  public CountFacetRecorder() {
+    // Has to be synchronizedList as we have one recorder per all slices.
+    perLeafValues = Collections.synchronizedList(new ArrayList<>());
+  }
+
+  /** Get count for provided ordinal. */
+  public int getCount(int ord) {
+    return values.get(ord);
+  }
+
+  @Override
+  public LeafFacetRecorder getLeafRecorder(LeafReaderContext context) {
+    IntIntHashMap leafValues = new IntIntHashMap();
+    perLeafValues.add(leafValues);

Review Comment:
   nit: It feels slightly more natural to me to keep track of a list of 
`CountLeafRecorder` instead of the underlying maps, and let the leaf recorders 
manage their own maps (then access them later in reduce). I don't feel that 
strongly though.



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/LongAggregationsFacetRecorder.java:
##########
@@ -0,0 +1,189 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.sandbox.facet.recorders;
+
+import static 
org.apache.lucene.sandbox.facet.ordinals.OrdinalIterator.NO_MORE_ORDS;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.internal.hppc.IntCursor;
+import org.apache.lucene.internal.hppc.IntObjectHashMap;
+import org.apache.lucene.sandbox.facet.misc.FacetRollup;
+import org.apache.lucene.sandbox.facet.ordinals.OrdinalIterator;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
+
+/**
+ * {@link FacetRecorder} that computes multiple long aggregations per facet.
+ *
+ * <p>TODO: [premature optimization idea] if instead of one array we keep 
aggregations in two
+ * LongVector (one for MAX aggregation and one for SUM) we can benefit from 
SIMD?
+ */
+public class LongAggregationsFacetRecorder implements FacetRecorder {
+
+  private IntObjectHashMap<long[]> values;
+  private List<IntObjectHashMap<long[]>> leafValues;

Review Comment:
   final?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/CountFacetRecorder.java:
##########
@@ -0,0 +1,151 @@
+/*
+ * 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.sandbox.facet.recorders;
+
+import static 
org.apache.lucene.sandbox.facet.ordinals.OrdinalIterator.NO_MORE_ORDS;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.internal.hppc.IntCursor;
+import org.apache.lucene.internal.hppc.IntIntHashMap;
+import org.apache.lucene.sandbox.facet.cutters.LeafFacetCutter;
+import org.apache.lucene.sandbox.facet.misc.FacetRollup;
+import org.apache.lucene.sandbox.facet.ordinals.OrdinalIterator;
+
+/**
+ * {@link FacetRecorder} to count facets.
+ *
+ * <p>TODO: add an option to keep counts in an array, to improve performance 
for facets with small
+ * number of ordinals e.g. range facets. Options: - {@link LeafFacetCutter} 
can inform {@link
+ * LeafFacetRecorder} about expected number of facet ordinals ({@link
+ * org.apache.lucene.sandbox.facet.FacetFieldCollector} can orchestrate that). 
If expeted facet ord
+ * number is below some threshold - use array instead of a map? - first 100/1k 
counts in array, the
+ * rest - in a map; the limit can also be provided in a constructor? It is 
similar to what
+ * LongValuesFacetCounts does today.
+ *
+ * <p>TODO: We can also consider collecting 2 (3, 4, ..., can be parametrizes) 
slices to a single
+ * sync map which can reduce thread contention compared to single sync map for 
all slices; at the
+ * same time there will be less work for reduce method. So far reduce wasn't a 
bottleneck for us,
+ * but it is definitely not free.
+ *
+ * <p>TODO: If we come back to some for of synchronized count maps, we should 
be more careful what
+ * we acquire locks for - we used to lock addTo method itself, but it could be 
faster if we only
+ * synchronized after computing the key's hash; or we can lock the entire map 
only if we need to
+ * insert key, and lock single key otherwise?
+ */
+public class CountFacetRecorder implements FacetRecorder {
+  IntIntHashMap values;
+  List<IntIntHashMap> perLeafValues;

Review Comment:
   `private`? Also, should `perLeafValues` be `final`?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/recorders/FacetRecorder.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.sandbox.facet.recorders;
+
+import java.io.IOException;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.sandbox.facet.cutters.LeafFacetCutter;
+import org.apache.lucene.sandbox.facet.misc.FacetRollup;
+import org.apache.lucene.sandbox.facet.ordinals.OrdinalIterator;
+
+/**
+ * Record data for each facet of each doc.
+ *
+ * <p>TODO: In the next iteration we can add an extra layer between 
FacetRecorder and
+ * LeafFacetRecorder, e.g. SliceFacetRecorder. The new layer will be created 
per {@link
+ * org.apache.lucene.search.Collector}, which means that collecting of 
multiple leafs (segments)
+ * within a slice is sequential and can be done to a single non-sync map to 
improve performance and
+ * reduce memory consumption. We already tried that, but didn't see any 
performance improvement.
+ * Given that it also makes lazy leaf recorder init in {@link
+ * org.apache.lucene.sandbox.facet.FacetFieldCollector} trickier, it was 
decided to rollback the
+ * initial attempt and try again later, in the next iteration.
+ */
+public interface FacetRecorder {
+  /** Get leaf recorder. */
+  LeafFacetRecorder getLeafRecorder(LeafReaderContext context) throws 
IOException;
+
+  /** Return next collected ordinal, or {@link LeafFacetCutter#NO_MORE_ORDS} */
+  OrdinalIterator recordedOrds();
+
+  /** True if there are no records */
+  boolean isEmpty();

Review Comment:
   I don't see any usage of this method right now. Should we remove it and 
re-introduce it later if a need arises?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/misc/LongValueFacetCutter.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.sandbox.facet.misc;

Review Comment:
   Would it make more sense to put this in the "cutters" sub-package?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/RangeOrdLabelBiMap.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.sandbox.facet.labels;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.lucene.facet.range.Range;
+import org.apache.lucene.facet.taxonomy.FacetLabel;
+
+/** {@link OrdLabelBiMap} for ranges. */
+public class RangeOrdLabelBiMap implements OrdLabelBiMap {
+
+  Range[] ranges;

Review Comment:
   Can this be `private final`?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/TaxonomyOrdLabelBiMap.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.sandbox.facet.labels;
+
+import java.io.IOException;
+import org.apache.lucene.facet.taxonomy.FacetLabel;
+import org.apache.lucene.facet.taxonomy.TaxonomyReader;
+
+/** Map taxonomy labels to ordinals. */
+public class TaxonomyOrdLabelBiMap implements OrdLabelBiMap {

Review Comment:
   Should we make this class final?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/ComparableUtils.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.sandbox.facet;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.sandbox.facet.ordinals.OrdToComparable;
+import org.apache.lucene.sandbox.facet.ordinals.OrdinalGetter;
+import org.apache.lucene.sandbox.facet.recorders.CountFacetRecorder;
+import org.apache.lucene.sandbox.facet.recorders.LongAggregationsFacetRecorder;
+import org.apache.lucene.util.InPlaceMergeSorter;
+
+/**
+ * Collection of static methods to provide most common comparables for sandbox 
faceting. You can
+ * also use it as an example for creating your own {@link OrdToComparable} to 
enable custom facets
+ * top-n and sorting.
+ */
+public class ComparableUtils {
+  private ComparableUtils() {}
+
+  private static class SkeletalOrdGetter implements OrdinalGetter {
+    int ord;
+
+    @Override
+    public int getOrd() {
+      return ord;
+    }
+  }
+
+  /** {@link OrdToComparable} that can be used to sort by ords (ascending). */
+  public static OrdToComparable<ComparableOrd> ordToComparableOrd() {
+    return new OrdToComparable<>() {
+      @Override
+      public ComparableOrd getComparable(int ord, ComparableOrd reuse) {
+        if (reuse == null) {
+          reuse = new ComparableOrd();
+        }
+        reuse.ord = ord;
+        return reuse;
+      }
+    };
+  }
+
+  /** Used for {@link #ordToComparableOrd} result. */
+  public static class ComparableOrd extends SkeletalOrdGetter implements 
Comparable<ComparableOrd> {
+    @Override
+    public int compareTo(ComparableOrd o) {
+      return Integer.compare(o.ord, ord);
+    }
+  }
+
+  /**
+   * {@link OrdToComparable} that can be used to sort ordinals by count 
(descending) with ord as a
+   * tie-break (ascending) using provided {@link CountFacetRecorder}.
+   */
+  public static OrdToComparable<ComparableIntOrd> ordToComparableCountOrd(
+      CountFacetRecorder recorder) {
+    return new OrdToComparable<>() {
+      @Override
+      public ComparableIntOrd getComparable(int ord, ComparableIntOrd reuse) {
+        if (reuse == null) {
+          reuse = new ComparableIntOrd();
+        }
+        reuse.ord = ord;
+        reuse.rank = recorder.getCount(ord);
+        return reuse;
+      }
+    };
+  }
+
+  /** Used for {@link #ordToComparableCountOrd} result. */
+  public static class ComparableIntOrd extends SkeletalOrdGetter
+      implements Comparable<ComparableIntOrd> {
+    private ComparableIntOrd() {}
+
+    private int rank;
+
+    @Override
+    public int compareTo(ComparableIntOrd o) {
+      int cmp = Integer.compare(rank, o.rank);
+      if (cmp == 0) {
+        cmp = Integer.compare(o.ord, ord);
+      }
+      return cmp;
+    }
+  }
+
+  /**
+   * {@link OrdToComparable} to sort ordinals by long aggregation (descending) 
with tie-break by
+   * count (descending) with ordinal as a tie-break (ascending) using provided 
{@link
+   * CountFacetRecorder} and {@link LongAggregationsFacetRecorder}.
+   */
+  public static OrdToComparable<ComparableLongIntOrd> 
ordToComparableRankCountOrd(
+      CountFacetRecorder countRecorder,
+      LongAggregationsFacetRecorder longAggregationsFacetRecorder,
+      int aggregationId) {
+    return new OrdToComparable<>() {
+      @Override
+      public ComparableLongIntOrd getComparable(int ord, ComparableLongIntOrd 
reuse) {
+        if (reuse == null) {
+          reuse = new ComparableLongIntOrd();
+        }
+        reuse.ord = ord;
+        reuse.secondaryRank = countRecorder.getCount(ord);
+        reuse.primaryRank = 
longAggregationsFacetRecorder.getRecordedValue(ord, aggregationId);
+        return reuse;
+      }
+    };
+  }
+
+  /** Used for {@link #ordToComparableRankCountOrd} result. */
+  public static class ComparableLongIntOrd extends SkeletalOrdGetter
+      implements Comparable<ComparableLongIntOrd> {
+    private ComparableLongIntOrd() {}
+    ;

Review Comment:
   Rogue semi-colon (tag: @mikemccand because I know he loves these)



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/RangeOrdLabelBiMap.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.sandbox.facet.labels;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.lucene.facet.range.Range;
+import org.apache.lucene.facet.taxonomy.FacetLabel;
+
+/** {@link OrdLabelBiMap} for ranges. */
+public class RangeOrdLabelBiMap implements OrdLabelBiMap {

Review Comment:
   Should we make this final?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/ComparableUtils.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.sandbox.facet;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.sandbox.facet.ordinals.OrdToComparable;
+import org.apache.lucene.sandbox.facet.ordinals.OrdinalGetter;
+import org.apache.lucene.sandbox.facet.recorders.CountFacetRecorder;
+import org.apache.lucene.sandbox.facet.recorders.LongAggregationsFacetRecorder;
+import org.apache.lucene.util.InPlaceMergeSorter;
+
+/**
+ * Collection of static methods to provide most common comparables for sandbox 
faceting. You can
+ * also use it as an example for creating your own {@link OrdToComparable} to 
enable custom facets
+ * top-n and sorting.
+ */
+public class ComparableUtils {
+  private ComparableUtils() {}
+
+  private static class SkeletalOrdGetter implements OrdinalGetter {
+    int ord;
+
+    @Override
+    public int getOrd() {
+      return ord;
+    }
+  }
+
+  /** {@link OrdToComparable} that can be used to sort by ords (ascending). */
+  public static OrdToComparable<ComparableOrd> ordToComparableOrd() {
+    return new OrdToComparable<>() {
+      @Override
+      public ComparableOrd getComparable(int ord, ComparableOrd reuse) {
+        if (reuse == null) {
+          reuse = new ComparableOrd();
+        }
+        reuse.ord = ord;
+        return reuse;
+      }
+    };
+  }
+
+  /** Used for {@link #ordToComparableOrd} result. */
+  public static class ComparableOrd extends SkeletalOrdGetter implements 
Comparable<ComparableOrd> {
+    @Override
+    public int compareTo(ComparableOrd o) {
+      return Integer.compare(o.ord, ord);
+    }
+  }
+
+  /**
+   * {@link OrdToComparable} that can be used to sort ordinals by count 
(descending) with ord as a
+   * tie-break (ascending) using provided {@link CountFacetRecorder}.
+   */
+  public static OrdToComparable<ComparableIntOrd> ordToComparableCountOrd(
+      CountFacetRecorder recorder) {
+    return new OrdToComparable<>() {
+      @Override
+      public ComparableIntOrd getComparable(int ord, ComparableIntOrd reuse) {
+        if (reuse == null) {
+          reuse = new ComparableIntOrd();
+        }
+        reuse.ord = ord;
+        reuse.rank = recorder.getCount(ord);
+        return reuse;
+      }
+    };
+  }
+
+  /** Used for {@link #ordToComparableCountOrd} result. */
+  public static class ComparableIntOrd extends SkeletalOrdGetter
+      implements Comparable<ComparableIntOrd> {
+    private ComparableIntOrd() {}
+
+    private int rank;
+
+    @Override
+    public int compareTo(ComparableIntOrd o) {
+      int cmp = Integer.compare(rank, o.rank);
+      if (cmp == 0) {
+        cmp = Integer.compare(o.ord, ord);
+      }
+      return cmp;
+    }
+  }
+
+  /**
+   * {@link OrdToComparable} to sort ordinals by long aggregation (descending) 
with tie-break by
+   * count (descending) with ordinal as a tie-break (ascending) using provided 
{@link
+   * CountFacetRecorder} and {@link LongAggregationsFacetRecorder}.
+   */
+  public static OrdToComparable<ComparableLongIntOrd> 
ordToComparableRankCountOrd(
+      CountFacetRecorder countRecorder,
+      LongAggregationsFacetRecorder longAggregationsFacetRecorder,
+      int aggregationId) {
+    return new OrdToComparable<>() {
+      @Override
+      public ComparableLongIntOrd getComparable(int ord, ComparableLongIntOrd 
reuse) {
+        if (reuse == null) {
+          reuse = new ComparableLongIntOrd();
+        }
+        reuse.ord = ord;
+        reuse.secondaryRank = countRecorder.getCount(ord);
+        reuse.primaryRank = 
longAggregationsFacetRecorder.getRecordedValue(ord, aggregationId);
+        return reuse;
+      }
+    };
+  }
+
+  /** Used for {@link #ordToComparableRankCountOrd} result. */
+  public static class ComparableLongIntOrd extends SkeletalOrdGetter
+      implements Comparable<ComparableLongIntOrd> {
+    private ComparableLongIntOrd() {}
+    ;
+
+    private int secondaryRank;
+    private long primaryRank;
+
+    @Override
+    public int compareTo(ComparableLongIntOrd o) {
+      int cmp = Long.compare(primaryRank, o.primaryRank);
+      if (cmp == 0) {
+        cmp = Integer.compare(secondaryRank, o.secondaryRank);
+        if (cmp == 0) {
+          cmp = Integer.compare(o.ord, ord);
+        }
+      }
+      return cmp;
+    }
+  }
+
+  /**
+   * Sort array of ordinals.
+   *
+   * <p>To get top-n ordinals use {@link
+   * org.apache.lucene.sandbox.facet.ordinals.TopnOrdinalIterator} instead.
+   *
+   * @param ordinals array of ordinals to sort
+   * @param ordToComparable defines sort order
+   */
+  public static <T extends Comparable<T> & OrdinalGetter> void sort(

Review Comment:
   nit: It doesn't matter if `T` extends `OrdinalGetter` does it? Can we remove 
that?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/ComparableUtils.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.sandbox.facet;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import org.apache.lucene.sandbox.facet.ordinals.OrdToComparable;
+import org.apache.lucene.sandbox.facet.ordinals.OrdinalGetter;
+import org.apache.lucene.sandbox.facet.recorders.CountFacetRecorder;
+import org.apache.lucene.sandbox.facet.recorders.LongAggregationsFacetRecorder;
+import org.apache.lucene.util.InPlaceMergeSorter;
+
+/**
+ * Collection of static methods to provide most common comparables for sandbox 
faceting. You can
+ * also use it as an example for creating your own {@link OrdToComparable} to 
enable custom facets
+ * top-n and sorting.
+ */
+public class ComparableUtils {

Review Comment:
   I really struggled with the naming of these static methods. I'm going to 
propose some alternatives here. Maybe there's something else even better? 
   
   * ordToComparableOrd -> byOrdinal
   * ordToComparableCountOrd -> byCount
   * ordToComparableRankCountOrd -> byAggregatedValue
   
   If you feel OK with a different naming strategy, here might be some 
alternative class names as well:
   * ComparableOrd -> ByOrdinalComparable
   * ComparableIntOrd -> ByCountComparable (yes, I know it's more generic than 
just count, but I can't really imagine other use-cases...)
   * ComparableLongIntOrd -> ByAggregatedValueComparable (same justification as 
above...)
   
   Also, as one last comment, should we make the comparator classes final? If 
users need custom comparison logic, I suspect they'll create their own factory 
and comparator classes from scratch. Not sure these need to be extension points.



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/RangeOrdLabelBiMap.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.sandbox.facet.labels;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.lucene.facet.range.Range;
+import org.apache.lucene.facet.taxonomy.FacetLabel;
+
+/** {@link OrdLabelBiMap} for ranges. */
+public class RangeOrdLabelBiMap implements OrdLabelBiMap {
+
+  Range[] ranges;
+
+  /** Constructor that takes array of Range objects as input * */
+  public RangeOrdLabelBiMap(Range[] inputRanges) {
+    ranges = inputRanges;
+  }
+
+  /** Constructor that takes List of Range objects as input * */
+  public RangeOrdLabelBiMap(List<? extends Range> inputRanges) {
+    ranges = inputRanges.toArray(new Range[0]);
+  }
+
+  @Override
+  public FacetLabel getLabel(int ordinal) throws IOException {
+    if (ordinal >= 0 && ordinal < ranges.length) {
+      return new FacetLabel(ranges[ordinal].label);
+    }
+    return null;
+  }
+
+  @Override
+  public FacetLabel[] getLabels(int[] ordinals) throws IOException {
+    FacetLabel[] facetLabels = new FacetLabel[ordinals.length];
+    for (int i = 0; i < ordinals.length; i++) {
+      facetLabels[i] = getLabel(ordinals[i]);
+    }
+    return facetLabels;
+  }
+
+  @Override
+  public int getOrd(FacetLabel label) {
+    throw new UnsupportedOperationException("Not yet supported for ranges");

Review Comment:
   Hmm... can we ever support this? Nothing stops the user from creating 
multiple ranges with identical labels right? (Not sure why they'd want to do 
that, but they could). I'm not sure anything forces labels and ordinals to be 
1:1 in this case?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/RangeFacetCutter.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.sandbox.facet.cutters.ranges;
+
+import org.apache.lucene.facet.range.DoubleRange;
+import org.apache.lucene.facet.range.LongRange;
+import org.apache.lucene.sandbox.facet.cutters.FacetCutter;
+import org.apache.lucene.util.NumericUtils;
+
+/** {@link FacetCutter} for ranges * */
+public abstract class RangeFacetCutter implements FacetCutter {
+  // TODO: we don't always have field, e.g. for custom DoubleValuesSources - 
let's remove it from
+  // here?
+  String field;
+
+  // TODO: make the constructor also take in requested value sources and ranges
+  // Ranges can be done now, we need to make a common interface for 
ValueSources
+  RangeFacetCutter(String field) {
+    this.field = field;
+  }
+
+  LongRange[] mapDoubleRangesToSortableLong(DoubleRange[] doubleRanges) {
+    LongRange[] longRanges = new LongRange[doubleRanges.length];
+    for (int i = 0; i < longRanges.length; i++) {
+      DoubleRange dr = doubleRanges[i];
+      longRanges[i] =
+          new LongRange(
+              dr.label,
+              NumericUtils.doubleToSortableLong(dr.min),
+              true,
+              NumericUtils.doubleToSortableLong(dr.max),
+              true);

Review Comment:
   Do we need to honor whether-or-not the double range was min/max 
inclusive/exclusive? It doesn't seem safe to hardcode this.



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/misc/LongValueFacetCutter.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.sandbox.facet.misc;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.lucene.facet.taxonomy.FacetLabel;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.internal.hppc.IntLongHashMap;
+import org.apache.lucene.internal.hppc.LongCursor;
+import org.apache.lucene.internal.hppc.LongHashSet;
+import org.apache.lucene.internal.hppc.LongIntHashMap;
+import org.apache.lucene.sandbox.facet.cutters.FacetCutter;
+import org.apache.lucene.sandbox.facet.cutters.LeafFacetCutter;
+import org.apache.lucene.sandbox.facet.labels.OrdLabelBiMap;
+
+/**
+ * {@link FacetCutter} and {@link OrdLabelBiMap} for distinct long values.
+ *
+ * <p>TODO: This class is quite inefficient. Will optimise later. TODO: add 
support for other value
+ * sources e.g: LongValues
+ */
+public class LongValueFacetCutter implements FacetCutter, OrdLabelBiMap {
+  private final String field;
+  // TODO: consider alternatives if this is a bottleneck
+  private final LongIntHashMap valueToOrdMap;
+  private final IntLongHashMap ordToValueMap;
+  private final AtomicInteger maxOrdinal;
+
+  /**
+   * Constructor.
+   *
+   * @param field field name to read long values from.
+   */
+  public LongValueFacetCutter(String field) {
+    this.field = field;
+    valueToOrdMap =
+        new LongIntHashMap() {
+          @Override
+          public synchronized boolean putIfAbsent(long key, int value) {
+            return super.putIfAbsent(key, value);
+          }
+        };
+    ordToValueMap = new IntLongHashMap();
+    maxOrdinal = new AtomicInteger(-1);
+  }
+
+  @Override
+  public LeafFacetCutter createLeafCutter(LeafReaderContext context) throws 
IOException {
+    SortedNumericDocValues docValues = 
DocValues.getSortedNumeric(context.reader(), field);
+    return new LeafFacetCutter() {
+      int currDoc = -1;
+      final LongHashSet valuesForDoc = new LongHashSet();
+      private Iterator<LongCursor> valuesCursor;
+
+      @Override
+      public boolean advanceExact(int doc) throws IOException {
+        if (doc < currDoc) {
+          return false;
+        }
+        if (doc == currDoc) {
+          return true;
+        }
+        valuesForDoc.clear();

Review Comment:
   nit: should we postpone clearning until we confirm advanceExact is 
successful?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/misc/LongValueFacetCutter.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.sandbox.facet.misc;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.lucene.facet.taxonomy.FacetLabel;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.internal.hppc.IntLongHashMap;
+import org.apache.lucene.internal.hppc.LongCursor;
+import org.apache.lucene.internal.hppc.LongHashSet;
+import org.apache.lucene.internal.hppc.LongIntHashMap;
+import org.apache.lucene.sandbox.facet.cutters.FacetCutter;
+import org.apache.lucene.sandbox.facet.cutters.LeafFacetCutter;
+import org.apache.lucene.sandbox.facet.labels.OrdLabelBiMap;
+
+/**
+ * {@link FacetCutter} and {@link OrdLabelBiMap} for distinct long values.
+ *
+ * <p>TODO: This class is quite inefficient. Will optimise later. TODO: add 
support for other value
+ * sources e.g: LongValues
+ */
+public class LongValueFacetCutter implements FacetCutter, OrdLabelBiMap {
+  private final String field;
+  // TODO: consider alternatives if this is a bottleneck
+  private final LongIntHashMap valueToOrdMap;
+  private final IntLongHashMap ordToValueMap;
+  private final AtomicInteger maxOrdinal;
+
+  /**
+   * Constructor.
+   *
+   * @param field field name to read long values from.
+   */
+  public LongValueFacetCutter(String field) {
+    this.field = field;
+    valueToOrdMap =
+        new LongIntHashMap() {
+          @Override
+          public synchronized boolean putIfAbsent(long key, int value) {
+            return super.putIfAbsent(key, value);
+          }
+        };
+    ordToValueMap = new IntLongHashMap();
+    maxOrdinal = new AtomicInteger(-1);
+  }
+
+  @Override
+  public LeafFacetCutter createLeafCutter(LeafReaderContext context) throws 
IOException {
+    SortedNumericDocValues docValues = 
DocValues.getSortedNumeric(context.reader(), field);
+    return new LeafFacetCutter() {
+      int currDoc = -1;
+      final LongHashSet valuesForDoc = new LongHashSet();
+      private Iterator<LongCursor> valuesCursor;
+
+      @Override
+      public boolean advanceExact(int doc) throws IOException {
+        if (doc < currDoc) {
+          return false;
+        }
+        if (doc == currDoc) {
+          return true;
+        }
+        valuesForDoc.clear();
+        if (docValues.advanceExact(doc)) {
+          int numValues = docValues.docValueCount();
+          for (int i = 0; i < numValues; i++) {

Review Comment:
   Why do we need to load all the values up-front? Would it be cleaner to 
iterate the underlying doc values with each call to `nextOrd`?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/misc/LongValueFacetCutter.java:
##########
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.sandbox.facet.misc;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.lucene.facet.taxonomy.FacetLabel;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedNumericDocValues;
+import org.apache.lucene.internal.hppc.IntLongHashMap;
+import org.apache.lucene.internal.hppc.LongCursor;
+import org.apache.lucene.internal.hppc.LongHashSet;
+import org.apache.lucene.internal.hppc.LongIntHashMap;
+import org.apache.lucene.sandbox.facet.cutters.FacetCutter;
+import org.apache.lucene.sandbox.facet.cutters.LeafFacetCutter;
+import org.apache.lucene.sandbox.facet.labels.OrdLabelBiMap;
+
+/**
+ * {@link FacetCutter} and {@link OrdLabelBiMap} for distinct long values.
+ *
+ * <p>TODO: This class is quite inefficient. Will optimise later. TODO: add 
support for other value
+ * sources e.g: LongValues
+ */
+public class LongValueFacetCutter implements FacetCutter, OrdLabelBiMap {
+  private final String field;
+  // TODO: consider alternatives if this is a bottleneck
+  private final LongIntHashMap valueToOrdMap;
+  private final IntLongHashMap ordToValueMap;
+  private final AtomicInteger maxOrdinal;
+
+  /**
+   * Constructor.
+   *
+   * @param field field name to read long values from.
+   */
+  public LongValueFacetCutter(String field) {
+    this.field = field;
+    valueToOrdMap =
+        new LongIntHashMap() {
+          @Override
+          public synchronized boolean putIfAbsent(long key, int value) {
+            return super.putIfAbsent(key, value);
+          }
+        };
+    ordToValueMap = new IntLongHashMap();
+    maxOrdinal = new AtomicInteger(-1);
+  }
+
+  @Override
+  public LeafFacetCutter createLeafCutter(LeafReaderContext context) throws 
IOException {
+    SortedNumericDocValues docValues = 
DocValues.getSortedNumeric(context.reader(), field);
+    return new LeafFacetCutter() {
+      int currDoc = -1;
+      final LongHashSet valuesForDoc = new LongHashSet();
+      private Iterator<LongCursor> valuesCursor;
+
+      @Override
+      public boolean advanceExact(int doc) throws IOException {
+        if (doc < currDoc) {
+          return false;
+        }
+        if (doc == currDoc) {
+          return true;
+        }
+        valuesForDoc.clear();
+        if (docValues.advanceExact(doc)) {
+          int numValues = docValues.docValueCount();
+          for (int i = 0; i < numValues; i++) {
+            long value = docValues.nextValue();
+            valueToOrdMap.putIfAbsent(value, maxOrdinal.incrementAndGet());

Review Comment:
   As far as I know, `LongIntHashMap` is not threadsafe. Don't we need it to be 
here if multiple leaves could be operating against it concurrently?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/RangeFacetCutter.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.sandbox.facet.cutters.ranges;
+
+import org.apache.lucene.facet.range.DoubleRange;
+import org.apache.lucene.facet.range.LongRange;
+import org.apache.lucene.sandbox.facet.cutters.FacetCutter;
+import org.apache.lucene.util.NumericUtils;
+
+/** {@link FacetCutter} for ranges * */
+public abstract class RangeFacetCutter implements FacetCutter {
+  // TODO: we don't always have field, e.g. for custom DoubleValuesSources - 
let's remove it from
+  // here?
+  String field;

Review Comment:
   As suggested by your comment, let's remove this? I don't see anything 
referencing it anyway?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/RangeFacetCutter.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.sandbox.facet.cutters.ranges;
+
+import org.apache.lucene.facet.range.DoubleRange;
+import org.apache.lucene.facet.range.LongRange;
+import org.apache.lucene.sandbox.facet.cutters.FacetCutter;
+import org.apache.lucene.util.NumericUtils;
+
+/** {@link FacetCutter} for ranges * */
+public abstract class RangeFacetCutter implements FacetCutter {
+  // TODO: we don't always have field, e.g. for custom DoubleValuesSources - 
let's remove it from
+  // here?
+  String field;
+
+  // TODO: make the constructor also take in requested value sources and ranges
+  // Ranges can be done now, we need to make a common interface for 
ValueSources
+  RangeFacetCutter(String field) {
+    this.field = field;
+  }
+
+  LongRange[] mapDoubleRangesToSortableLong(DoubleRange[] doubleRanges) {

Review Comment:
   This is only used by `DoubleRangeFacetCutter` right? Should we move it into 
that class? And then is there any need for this abstract class?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/FacetFieldCollectorManager.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.sandbox.facet;
+
+import java.io.IOException;
+import java.util.Collection;
+import org.apache.lucene.sandbox.facet.cutters.FacetCutter;
+import org.apache.lucene.sandbox.facet.misc.FacetRollup;
+import org.apache.lucene.sandbox.facet.recorders.FacetRecorder;
+import org.apache.lucene.search.CollectorManager;
+
+/**
+ * Collector manager for {@link FacetFieldCollector}. Returns the same 
extension of {@link
+ * FacetRecorder} that was used to collect results.
+ */
+public final class FacetFieldCollectorManager<V extends FacetRecorder>
+    implements CollectorManager<FacetFieldCollector, V> {
+
+  private final FacetCutter facetCutter;
+  private final V facetRecorder;
+  private final FacetRollup facetRollup;
+
+  /** Create collector for a cutter + recorder pair */
+  public FacetFieldCollectorManager(

Review Comment:
   Assuming I'm following the code flow properly (I may not be), it looks like 
`FacetRollup` is only needed by `FacetRecorder` (to know what to rollup when 
reducing). I wonder if it would be a little cleaner to have facet recorders 
keep track of a `FacetRollup` instance during construction instead of having it 
passed into `reduce`, and then getting rid of the need for collectors/managers 
to know anything about it? Is there ever a need to have the same recorder use 
different rollup strategies during reduce? If so, I guess we need to keep it 
this way. But if not, maybe we can tighten this up a bit?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/LongRangeFacetCutter.java:
##########
@@ -0,0 +1,413 @@
+/*
+ * 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.sandbox.facet.cutters.ranges;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+import org.apache.lucene.facet.MultiLongValues;
+import org.apache.lucene.facet.MultiLongValuesSource;
+import org.apache.lucene.facet.range.LongRange;
+import org.apache.lucene.sandbox.facet.cutters.FacetCutter;
+import org.apache.lucene.sandbox.facet.cutters.LeafFacetCutter;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
+
+/** {@link RangeFacetCutter} for ranges of long values. It's based on 
LongRangeCounter class. */
+public abstract class LongRangeFacetCutter extends RangeFacetCutter {
+
+  MultiLongValuesSource valuesSource;
+  LongValuesSource singleValues; // TODO: refactor - weird that we have both 
multi and single here.
+  LongRangeAndPos[] sortedRanges;
+
+  int requestedRangeCount;
+
+  List<InclusiveRange> elementaryIntervals;
+
+  /** elementary interval boundaries used for efficient counting (bsearch to 
find interval) */
+  long[] boundaries;
+
+  int[] pos;
+
+  // Default interval position, when elementary interval is mapped to this 
interval
+  // it is skipped.
+  static final int SKIP_INTERVAL_POSITION = -1;
+
+  /** Create {@link FacetCutter} for provided value source and long ranges. */
+  public static LongRangeFacetCutter create(

Review Comment:
   Why are we exposing this publicly? Users wouldn't ever pass single and multi 
valued sources would they?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/LongRangeFacetCutter.java:
##########
@@ -0,0 +1,413 @@
+/*
+ * 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.sandbox.facet.cutters.ranges;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+import org.apache.lucene.facet.MultiLongValues;
+import org.apache.lucene.facet.MultiLongValuesSource;
+import org.apache.lucene.facet.range.LongRange;
+import org.apache.lucene.sandbox.facet.cutters.FacetCutter;
+import org.apache.lucene.sandbox.facet.cutters.LeafFacetCutter;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
+
+/** {@link RangeFacetCutter} for ranges of long values. It's based on 
LongRangeCounter class. */
+public abstract class LongRangeFacetCutter extends RangeFacetCutter {
+
+  MultiLongValuesSource valuesSource;
+  LongValuesSource singleValues; // TODO: refactor - weird that we have both 
multi and single here.
+  LongRangeAndPos[] sortedRanges;
+
+  int requestedRangeCount;
+
+  List<InclusiveRange> elementaryIntervals;
+
+  /** elementary interval boundaries used for efficient counting (bsearch to 
find interval) */
+  long[] boundaries;
+
+  int[] pos;

Review Comment:
   Let's make all of these `final`



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/labels/RangeOrdLabelBiMap.java:
##########
@@ -0,0 +1,65 @@
+/*
+ * 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.sandbox.facet.labels;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.lucene.facet.range.Range;
+import org.apache.lucene.facet.taxonomy.FacetLabel;
+
+/** {@link OrdLabelBiMap} for ranges. */
+public class RangeOrdLabelBiMap implements OrdLabelBiMap {
+
+  Range[] ranges;
+
+  /** Constructor that takes array of Range objects as input * */
+  public RangeOrdLabelBiMap(Range[] inputRanges) {
+    ranges = inputRanges;
+  }
+
+  /** Constructor that takes List of Range objects as input * */
+  public RangeOrdLabelBiMap(List<? extends Range> inputRanges) {
+    ranges = inputRanges.toArray(new Range[0]);
+  }
+
+  @Override
+  public FacetLabel getLabel(int ordinal) throws IOException {
+    if (ordinal >= 0 && ordinal < ranges.length) {
+      return new FacetLabel(ranges[ordinal].label);
+    }
+    return null;
+  }
+
+  @Override
+  public FacetLabel[] getLabels(int[] ordinals) throws IOException {
+    FacetLabel[] facetLabels = new FacetLabel[ordinals.length];
+    for (int i = 0; i < ordinals.length; i++) {
+      facetLabels[i] = getLabel(ordinals[i]);
+    }
+    return facetLabels;
+  }
+
+  @Override
+  public int getOrd(FacetLabel label) {
+    throw new UnsupportedOperationException("Not yet supported for ranges");

Review Comment:
   I bring this up not to be nit-picky. I wonder if we should have two 
interfaces? One for ordToLabel and one for labelToOrd?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/LongRangeFacetCutter.java:
##########
@@ -0,0 +1,413 @@
+/*
+ * 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.sandbox.facet.cutters.ranges;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+import org.apache.lucene.facet.MultiLongValues;
+import org.apache.lucene.facet.MultiLongValuesSource;
+import org.apache.lucene.facet.range.LongRange;
+import org.apache.lucene.sandbox.facet.cutters.FacetCutter;
+import org.apache.lucene.sandbox.facet.cutters.LeafFacetCutter;
+import org.apache.lucene.search.LongValues;
+import org.apache.lucene.search.LongValuesSource;
+
+/** {@link RangeFacetCutter} for ranges of long values. It's based on 
LongRangeCounter class. */
+public abstract class LongRangeFacetCutter extends RangeFacetCutter {
+
+  MultiLongValuesSource valuesSource;
+  LongValuesSource singleValues; // TODO: refactor - weird that we have both 
multi and single here.
+  LongRangeAndPos[] sortedRanges;
+
+  int requestedRangeCount;
+
+  List<InclusiveRange> elementaryIntervals;
+
+  /** elementary interval boundaries used for efficient counting (bsearch to 
find interval) */
+  long[] boundaries;
+
+  int[] pos;
+
+  // Default interval position, when elementary interval is mapped to this 
interval
+  // it is skipped.
+  static final int SKIP_INTERVAL_POSITION = -1;
+
+  /** Create {@link FacetCutter} for provided value source and long ranges. */
+  public static LongRangeFacetCutter create(
+      String field,
+      MultiLongValuesSource longValuesSource,
+      LongValuesSource singleLongValuesSource,
+      LongRange[] longRanges) {
+    if (areOverlappingRanges(longRanges)) {
+      return new OverlappingLongRangeFacetCutter(
+          field, longValuesSource, singleLongValuesSource, longRanges);
+    }
+    return new NonOverlappingLongRangeFacetCutter(
+        field, longValuesSource, singleLongValuesSource, longRanges);
+  }
+
+  public static LongRangeFacetCutter create(
+      String field, MultiLongValuesSource longValuesSource, LongRange[] 
longRanges) {
+    return create(field, longValuesSource, null, longRanges);
+  }
+
+  // caller handles conversion of Doubles and DoubleRange to Long and LongRange
+  // ranges need not be sorted
+  LongRangeFacetCutter(
+      String field,
+      MultiLongValuesSource longValuesSource,
+      LongValuesSource singleLongValuesSource,
+      LongRange[] longRanges) {
+    super(field);
+    valuesSource = longValuesSource;
+    if (singleLongValuesSource != null) {
+      singleValues = singleLongValuesSource;
+    } else {
+      singleValues = MultiLongValuesSource.unwrapSingleton(valuesSource);
+    }
+
+    sortedRanges = new LongRangeAndPos[longRanges.length];
+    requestedRangeCount = longRanges.length;
+
+    for (int i = 0; i < longRanges.length; i++) {
+      sortedRanges[i] = new LongRangeAndPos(longRanges[i], i);
+    }
+
+    Arrays.sort(this.sortedRanges, Comparator.comparingLong(r -> r.range.min));
+    elementaryIntervals = buildElementaryIntervals();
+
+    // Keep track of elementary interval boundary ends (for binary search) 
along with the requested
+    // range they map back to (and -1 when they map to a "gap" range in case 
of ExclusiveRanges):
+    boundaries = new long[elementaryIntervals.size()];
+    pos = new int[elementaryIntervals.size()];
+    Arrays.fill(pos, SKIP_INTERVAL_POSITION);
+    int currRange = 0;
+    for (int i = 0; i < boundaries.length; i++) {
+      boundaries[i] = elementaryIntervals.get(i).end;
+      if (currRange < sortedRanges.length) {
+        LongRangeAndPos curr = sortedRanges[currRange];
+        if (boundaries[i] == curr.range.max) {
+          pos[i] = curr.pos;
+          currRange++;
+        }
+      }
+    }
+  }
+
+  abstract List<InclusiveRange> buildElementaryIntervals();
+
+  private static boolean areOverlappingRanges(LongRange[] ranges) {
+    if (ranges.length == 0) {
+      return false;
+    }
+
+    // Copy before sorting so we don't mess with the caller's original ranges:
+    // TODO: We're going to do this again in the constructor. Can't we come up 
with a clever way to
+    // avoid doing it twice?
+    LongRange[] sortedRanges = new LongRange[ranges.length];
+    System.arraycopy(ranges, 0, sortedRanges, 0, ranges.length);
+    Arrays.sort(sortedRanges, Comparator.comparingLong(r -> r.min));
+
+    long previousMax = sortedRanges[0].max;
+    for (int i = 1; i < sortedRanges.length; i++) {
+      // Ranges overlap if the next min is <= the previous max (note that 
LongRange models
+      // closed ranges, so equal limit points are considered overlapping):
+      if (sortedRanges[i].min <= previousMax) {
+        return true;
+      }
+      previousMax = sortedRanges[i].max;
+    }
+
+    return false;
+  }
+
+  abstract static class LongRangeMultivaluedLeafFacetCutter implements 
LeafFacetCutter {
+    final MultiLongValues multiLongValues;
+    final long[] boundaries;
+    final int[] pos;
+
+    final int requestedRangeCount;
+
+    // int currentDoc = -1;
+
+    final IntervalTracker elementaryIntervalTracker;
+
+    // TODO: we need it only for overlapping ranges, should not handle it in 
advanceExact for
+    // exclusive ranges.
+    IntervalTracker requestedIntervalTracker;
+
+    LongRangeMultivaluedLeafFacetCutter(
+        MultiLongValues longValues, long[] boundaries, int[] pos, int 
requestedRangeCount) {
+      this.multiLongValues = longValues;
+      this.boundaries = boundaries;
+      this.pos = pos;
+      this.requestedRangeCount = requestedRangeCount;
+      elementaryIntervalTracker = new 
IntervalTracker.MultiIntervalTracker(boundaries.length);
+    }
+
+    @Override
+    public boolean advanceExact(int doc) throws IOException {
+      if (multiLongValues.advanceExact(doc) == false) {
+        return false;
+      }
+
+      elementaryIntervalTracker.clear();
+
+      if (requestedIntervalTracker != null) {
+        requestedIntervalTracker.clear();
+      }
+
+      long numValues = multiLongValues.getValueCount();
+
+      int lastIntervalSeen = -1;
+
+      for (int i = 0; i < numValues; i++) {
+        lastIntervalSeen = processValue(multiLongValues.nextValue(), 
lastIntervalSeen);
+        if (lastIntervalSeen >= 0 && lastIntervalSeen < boundaries.length) {
+          elementaryIntervalTracker.set(lastIntervalSeen);
+        }
+        if (lastIntervalSeen == boundaries.length - 1) {
+          // we've already reached the end of all possible intervals for this 
doc
+          break;
+        }
+      }
+      maybeRollUp(requestedIntervalTracker);
+
+      elementaryIntervalTracker.freeze();
+
+      if (requestedIntervalTracker != null) {
+        requestedIntervalTracker.freeze();
+      }
+
+      return true;
+    }
+
+    // Returns the value of the interval v belongs or lastIntervalSeen
+    // if no processing is done, it returns the lastIntervalSeen
+    private int processValue(long v, int lastIntervalSeen) {
+      int lo = 0, hi = boundaries.length - 1;
+      ;
+
+      if (lastIntervalSeen != -1) {
+        // this is the multivalued doc case, we need to set lo correctly
+        if (v <= boundaries[lastIntervalSeen]) {
+          // we've already counted something for this interval and doc
+          // we don't need to process v
+          return lastIntervalSeen;
+        }
+
+        lo = lastIntervalSeen + 1;
+        if (lo == boundaries.length) {
+          // we've already counted the last elementary interval. If so, 
there's nothing
+          // else to count for this doc
+          // TODO: does it make sense to return something else?
+          return lastIntervalSeen;
+        }
+      }
+      int lowerBound = lo;
+
+      while (true) {
+        int mid = (lo + hi) >>> 1;
+        if (v <= boundaries[mid]) {
+          if (mid == lowerBound) {
+            return mid;
+          } else {
+            hi = mid - 1;
+          }
+        } else if (v > boundaries[mid + 1]) {
+          lo = mid + 1;
+        } else {
+          return mid + 1;
+        }
+      }
+    }
+
+    void maybeRollUp(IntervalTracker rollUpInto) {}
+  }
+
+  abstract static class LongRangeSingleValuedLeafFacetCutter implements 
LeafFacetCutter {
+    final LongValues longValues;
+    final long[] boundaries;
+    final int[] pos;
+
+    final int requestedRangeCount;
+    final IntervalTracker elementaryIntervalTracker;
+
+    IntervalTracker requestedIntervalTracker;
+
+    LongRangeSingleValuedLeafFacetCutter(
+        LongValues longValues, long[] boundaries, int[] pos, int 
requestedRangeCount) {
+      this.longValues = longValues;
+      this.boundaries = boundaries;
+      this.pos = pos;
+      this.requestedRangeCount = requestedRangeCount;
+      elementaryIntervalTracker = new IntervalTracker.SingleIntervalTracker();
+    }
+
+    @Override
+    public boolean advanceExact(int doc) throws IOException {
+      if (longValues.advanceExact(doc) == false) {
+        return false;
+      }
+
+      elementaryIntervalTracker.clear();
+
+      if (requestedIntervalTracker != null) {
+        requestedIntervalTracker.clear();
+      }
+
+      int lastIntervalSeen = -1;
+
+      lastIntervalSeen = processValue(longValues.longValue(), 
lastIntervalSeen);
+      elementaryIntervalTracker.set(lastIntervalSeen);
+
+      maybeRollUp(requestedIntervalTracker);
+
+      // if (elementaryIntervalTracker != null) {
+      elementaryIntervalTracker.freeze();
+      // }
+      if (requestedIntervalTracker != null) {
+        requestedIntervalTracker.freeze();
+      }
+
+      return true;
+    }
+
+    // Returns the value of the interval v belongs or lastIntervalSeen
+    // if no processing is done, it returns the lastIntervalSeen
+    // TODO: dedup with multi valued?
+    private int processValue(long v, int lastIntervalSeen) {
+      int lo = 0, hi = boundaries.length - 1;
+      ;
+
+      if (lastIntervalSeen != -1) {
+        // this is the multivalued doc case, we need to set lo correctly
+        if (v <= boundaries[lastIntervalSeen]) {
+          // we've already counted something for this interval and doc
+          // we don't need to process v
+          return lastIntervalSeen;
+        }
+
+        lo = lastIntervalSeen + 1;
+        if (lo == boundaries.length) {
+          // we've already counted the last elementary interval. If so, 
there's nothing
+          // else to count for this doc
+          // TODO: does it make sense to return something else?
+          return lastIntervalSeen;
+        }
+      }
+      int lowerBound = lo;
+
+      while (true) {
+        int mid = (lo + hi) >>> 1;
+        if (v <= boundaries[mid]) {
+          if (mid == lowerBound) {
+            return mid;
+          } else {
+            hi = mid - 1;
+          }
+        } else if (v > boundaries[mid + 1]) {
+          lo = mid + 1;
+        } else {
+          return mid + 1;
+        }
+      }
+    }
+
+    void maybeRollUp(IntervalTracker rollUpInto) {}
+  }
+
+  static final class LongRangeAndPos {
+    private final LongRange range;
+    private final int pos;
+
+    LongRangeAndPos(LongRange range, int pos) {
+      this.range = range;
+      this.pos = pos;
+    }
+
+    LongRange range() {
+      return range;
+    }
+
+    int pos() {
+      return pos;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (obj == this) return true;
+      if (obj == null || obj.getClass() != this.getClass()) return false;
+      var that = (LongRangeAndPos) obj;
+      return Objects.equals(this.range, that.range) && this.pos == that.pos;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(range, pos);
+    }
+
+    @Override
+    public String toString() {
+      return "LongRangeAndPos[" + "range=" + range + ", " + "pos=" + pos + ']';
+    }
+  }
+
+  /**
+   * Similar to InclusiveRange from LongRangeCounter.
+   *
+   * <p>TODO: dedup
+   */
+  static final class InclusiveRange {

Review Comment:
   Let's use `record` here?



##########
lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/ordinals/OrdToComparable.java:
##########
@@ -0,0 +1,36 @@
+/*
+ * 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.sandbox.facet.ordinals;
+
+/**
+ * Generates {@link Comparable} for provided ordinal. For example it can be 
used to find topN facet
+ * ordinals.
+ *
+ * @param <T> something ordinals can be compared by.
+ */
+public interface OrdToComparable<T extends Comparable<T>> {

Review Comment:
   I really struggled with this name when trying to understand the code/API. 
What if we use "supplier" or "factory" terminology? What about something like 
"ComparableSupplier" or "ComparableFactory" or something? 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org
For additional commands, e-mail: issues-h...@lucene.apache.org

Reply via email to