mikemccand commented on code in PR #13568: URL: https://github.com/apache/lucene/pull/13568#discussion_r1698434531
########## lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/LongValueFacetCutter.java: ########## @@ -0,0 +1,173 @@ +/* + * 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; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.IntSupplier; +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.LongIntHashMap; +import org.apache.lucene.sandbox.facet.labels.OrdToLabel; + +/** + * {@link FacetCutter} and {@link OrdToLabel} for distinct long values. + * + * <p>TODO: This class is quite inefficient. Will optimise later. + * + * <p>TODO: add support for other value sources e.g: LongValues + */ +public final class LongValueFacetCutter implements FacetCutter, OrdToLabel { + private final String field; + // TODO: consider alternatives if this is a bottleneck + private final LongIntHashMapSyncCompute valueToOrdMap; + private 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 LongIntHashMapSyncCompute(); + ordToValueMap = null; + maxOrdinal = new AtomicInteger(-1); + } + + @Override + public LeafFacetCutter createLeafCutter(LeafReaderContext context) throws IOException { + SortedNumericDocValues docValues = DocValues.getSortedNumeric(context.reader(), field); + return new LeafFacetCutter() { + int docValueCount; + long lastDocValue; + int docValueCursor; + + @Override + public boolean advanceExact(int doc) throws IOException { + if (docValues.advanceExact(doc)) { + docValueCount = docValues.docValueCount(); + docValueCursor = 0; + return true; + } + return false; + } + + @Override + public int nextOrd() throws IOException { + while (docValueCursor++ < docValueCount) { + long value = docValues.nextValue(); + // SortedNumericDocValues can have duplicates, but values are sorted, so we only need to + // check previous value to remove duplicates + if (docValueCursor == 1 || value != lastDocValue) { + lastDocValue = value; + return valueToOrdMap.computeIfAbsent(value, maxOrdinal::incrementAndGet); + } + } + return NO_MORE_ORDS; + } + }; + } + + @Override + public FacetLabel getLabel(int ordinal) { + if (ordToValueMap == null) { + buildOrdToValueMap(); + } + if (ordToValueMap.containsKey(ordinal)) { + return new FacetLabel(String.valueOf(ordToValueMap.get(ordinal))); + } + assert false + : "ordinal=" + + ordinal + + ", ordToValueMap.size=" + + ordToValueMap.size() + + ", valueToOrdMap.size=" + + valueToOrdMap.size(); + return null; + } + + /** + * Get value by ordinal. Should only be called after collection phase. + * + * <p>TODO: we need it to tie break sort by value. Alternatively we can sort by label (then we + * don't need this method), but we would have to convert FacetLabel to "long" to have the same + * order... Overall, it is probably not important to tie break by value, and we can tie break by + * ord same as for other facets; but for now we don't want to change results order just in case. + * + * @param ordinal facet ordinal. + * @return long value + */ + public long getValue(int ordinal) { + // TODO: do we want to create #finish method that called by #reduce to build the map? + if (ordToValueMap == null) { + buildOrdToValueMap(); + } + return ordToValueMap.get(ordinal); + } + + private void buildOrdToValueMap() { + ordToValueMap = new IntLongHashMap(valueToOrdMap.size()); + for (LongIntHashMap.LongIntCursor cursor : valueToOrdMap) { + ordToValueMap.put(cursor.value, cursor.key); + } + } + + @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; + } + + private static class LongIntHashMapSyncCompute extends LongIntHashMap { Review Comment: Maybe add a javadoc (even though it's a private class) explaining what this is doing? It's adding locking during the `computeIfAbsent` such that only one thread may compute the value even across different keys? ########## lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/LongValueFacetCutter.java: ########## @@ -0,0 +1,173 @@ +/* + * 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; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.locks.Lock; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.function.IntSupplier; +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.LongIntHashMap; +import org.apache.lucene.sandbox.facet.labels.OrdToLabel; + +/** + * {@link FacetCutter} and {@link OrdToLabel} for distinct long values. + * + * <p>TODO: This class is quite inefficient. Will optimise later. + * + * <p>TODO: add support for other value sources e.g: LongValues + */ +public final class LongValueFacetCutter implements FacetCutter, OrdToLabel { + private final String field; + // TODO: consider alternatives if this is a bottleneck + private final LongIntHashMapSyncCompute valueToOrdMap; + private 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 LongIntHashMapSyncCompute(); + ordToValueMap = null; + maxOrdinal = new AtomicInteger(-1); + } + + @Override + public LeafFacetCutter createLeafCutter(LeafReaderContext context) throws IOException { + SortedNumericDocValues docValues = DocValues.getSortedNumeric(context.reader(), field); + return new LeafFacetCutter() { + int docValueCount; + long lastDocValue; + int docValueCursor; + + @Override + public boolean advanceExact(int doc) throws IOException { + if (docValues.advanceExact(doc)) { + docValueCount = docValues.docValueCount(); + docValueCursor = 0; + return true; + } + return false; + } + + @Override + public int nextOrd() throws IOException { + while (docValueCursor++ < docValueCount) { + long value = docValues.nextValue(); + // SortedNumericDocValues can have duplicates, but values are sorted, so we only need to + // check previous value to remove duplicates + if (docValueCursor == 1 || value != lastDocValue) { + lastDocValue = value; + return valueToOrdMap.computeIfAbsent(value, maxOrdinal::incrementAndGet); + } + } + return NO_MORE_ORDS; + } + }; + } + + @Override + public FacetLabel getLabel(int ordinal) { + if (ordToValueMap == null) { + buildOrdToValueMap(); + } + if (ordToValueMap.containsKey(ordinal)) { + return new FacetLabel(String.valueOf(ordToValueMap.get(ordinal))); + } + assert false + : "ordinal=" + + ordinal + + ", ordToValueMap.size=" + + ordToValueMap.size() + + ", valueToOrdMap.size=" + + valueToOrdMap.size(); + return null; + } + + /** + * Get value by ordinal. Should only be called after collection phase. + * + * <p>TODO: we need it to tie break sort by value. Alternatively we can sort by label (then we + * don't need this method), but we would have to convert FacetLabel to "long" to have the same + * order... Overall, it is probably not important to tie break by value, and we can tie break by + * ord same as for other facets; but for now we don't want to change results order just in case. + * + * @param ordinal facet ordinal. + * @return long value + */ + public long getValue(int ordinal) { + // TODO: do we want to create #finish method that called by #reduce to build the map? + if (ordToValueMap == null) { + buildOrdToValueMap(); + } + return ordToValueMap.get(ordinal); + } + + private void buildOrdToValueMap() { + ordToValueMap = new IntLongHashMap(valueToOrdMap.size()); + for (LongIntHashMap.LongIntCursor cursor : valueToOrdMap) { + ordToValueMap.put(cursor.value, cursor.key); + } + } + + @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; + } + + private static class LongIntHashMapSyncCompute extends LongIntHashMap { + private final ReentrantReadWriteLock rwl = new ReentrantReadWriteLock(); + private final Lock r = rwl.readLock(); + private final Lock w = rwl.writeLock(); + + public int computeIfAbsent(long key, IntSupplier valueSupplier) { + r.lock(); + int value = super.getOrDefault(key, -1); + r.unlock(); Review Comment: Paranoia: should we put this in a `finally` clause too? ########## lucene/facet/src/java/org/apache/lucene/facet/MultiDoubleValuesSource.java: ########## @@ -118,6 +119,66 @@ public final MultiLongValuesSource toMultiLongValuesSource() { return new LongDoubleValuesSource(this); } + /** Convert to a SortableMultiLongValuesSource. * */ + public final SortableMultiLongValuesSource toSortableMultiLongValuesSource() { + return new SortableMultiLongValuesSource(this); + } + + /** Convert inner double values to sortable long using NumericUtils.doubleToSortableLong */ + public static class SortableMultiLongValuesSource extends MultiLongValuesSource { + + MultiDoubleValuesSource inner; + + SortableMultiLongValuesSource(MultiDoubleValuesSource inner) { + this.inner = inner; + } + + @Override + public boolean isCacheable(LeafReaderContext ctx) { + return inner.isCacheable(ctx); + } + + @Override + public MultiLongValues getValues(LeafReaderContext ctx) throws IOException { + MultiDoubleValues doubleValues = inner.getValues(ctx); + + return new MultiLongValues() { + @Override + public long getValueCount() { + return doubleValues.getValueCount(); + } + + @Override + public long nextValue() throws IOException { + return NumericUtils.doubleToSortableLong(doubleValues.nextValue()); + } + + @Override + public boolean advanceExact(int doc) throws IOException { + return doubleValues.advanceExact(doc); + } + }; + } + + @Override + public int hashCode() { + return Objects.hash(inner); Review Comment: > Why not `inner.hashCode()`? +1, and let's `Objects.requireNonNull` of `inner` in the ctor? ########## lucene/facet/src/java/org/apache/lucene/facet/DrillSidewaysQuery.java: ########## @@ -45,58 +45,26 @@ class DrillSidewaysQuery extends Query { final Query baseQuery; - final FacetsCollectorManager drillDownCollectorManager; - final FacetsCollectorManager[] drillSidewaysCollectorManagers; - final List<FacetsCollector> managedDrillDownCollectors; - final List<FacetsCollector[]> managedDrillSidewaysCollectors; + final CollectorOwner<?, ?> drillDownCollectorOwner; + final List<CollectorOwner<?, ?>> drillSidewaysCollectorOwners; final Query[] drillDownQueries; final boolean scoreSubDocsAtOnce; /** * Construct a new {@code DrillSidewaysQuery} that will create new {@link FacetsCollector}s for - * each {@link LeafReaderContext} using the provided {@link FacetsCollectorManager}s. The caller - * can access the created {@link FacetsCollector}s through {@link #managedDrillDownCollectors} and - * {@link #managedDrillSidewaysCollectors}. + * each {@link LeafReaderContext} using the provided {@link FacetsCollectorManager}s. */ DrillSidewaysQuery( Query baseQuery, - FacetsCollectorManager drillDownCollectorManager, - FacetsCollectorManager[] drillSidewaysCollectorManagers, - Query[] drillDownQueries, - boolean scoreSubDocsAtOnce) { - // Note that the "managed" facet collector lists are synchronized here since bulkScorer() - // can be invoked concurrently and needs to remain thread-safe. We're OK with synchronizing - // on the whole list as contention is expected to remain very low: - this( - baseQuery, - drillDownCollectorManager, - drillSidewaysCollectorManagers, - Collections.synchronizedList(new ArrayList<>()), - Collections.synchronizedList(new ArrayList<>()), - drillDownQueries, - scoreSubDocsAtOnce); - } - - /** - * Needed for {@link Query#rewrite(IndexSearcher)}. Ensures the same "managed" lists get used - * since {@link DrillSideways} accesses references to these through the original {@code - * DrillSidewaysQuery}. - */ - private DrillSidewaysQuery( - Query baseQuery, - FacetsCollectorManager drillDownCollectorManager, - FacetsCollectorManager[] drillSidewaysCollectorManagers, - List<FacetsCollector> managedDrillDownCollectors, - List<FacetsCollector[]> managedDrillSidewaysCollectors, + CollectorOwner<?, ?> drillDownCollectorOwner, Review Comment: Since this is an API break (we are deleting a ctor), could you add an entry to `main`'s `MIGRATE.txt` explaining how existing `DrillSidewaysQuery` users would migrate to the `CollectorOwner`? Also, maybe add `@lucene.experimental` to this class's javadocs to make it clear the API is still "malleable" even in feature releases (9.12)? This is technically cheating, since it is a public class and was released already and we are retroactively claiming we can change the API, but this is a very rarely used class among Lucene users (my impression) so I'm OK with breaking the rule in this case. ########## lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/LongRangeFacetCutter.java: ########## @@ -0,0 +1,354 @@ +/* + * 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 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 FacetCutter} for ranges of long values. It's based on LongRangeCounter class. */ +public abstract class LongRangeFacetCutter implements FacetCutter { + + final MultiLongValuesSource valuesSource; + final LongValuesSource + singleValues; // TODO: refactor - weird that we have both multi and single here. Review Comment: Remove the newline here? Or did spotless insert that? ########## lucene/facet/src/java/org/apache/lucene/facet/taxonomy/FacetLabel.java: ########## @@ -178,6 +178,11 @@ public FacetLabel subpath(final int length) { } } + /** Get the last component. */ + public String lastComponent() { + return components[components.length - 1]; Review Comment: Maybe check that `components.length > 0` first, and throw a reasonable exception? ########## lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java: ########## @@ -115,6 +116,69 @@ public final LongValuesSource toLongValuesSource() { return new LongDoubleValuesSource(this); } + public final LongValuesSource toSortableLongDoubleValuesSource() { + return new SortableLongDoubleValuesSource(this); + } + + private static class SortableLongDoubleValuesSource extends LongValuesSource { + + private final DoubleValuesSource inner; + + private SortableLongDoubleValuesSource(DoubleValuesSource inner) { + this.inner = inner; Review Comment: Let's explicitly not allow `null` here? E.g. `this.inner = Objects.requireNonNull(inner)`? ########## lucene/sandbox/src/java/org/apache/lucene/sandbox/facet/cutters/ranges/LongRangeFacetCutter.java: ########## @@ -0,0 +1,354 @@ +/* + * 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 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 FacetCutter} for ranges of long values. It's based on LongRangeCounter class. */ +public abstract class LongRangeFacetCutter implements FacetCutter { + + final MultiLongValuesSource valuesSource; + final LongValuesSource + singleValues; // TODO: refactor - weird that we have both multi and single here. + final LongRangeAndPos[] sortedRanges; + + final int requestedRangeCount; + + final List<InclusiveRange> elementaryIntervals; + + /** elementary interval boundaries used for efficient counting (bsearch to find interval) */ + final long[] boundaries; + + final 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. */ + static LongRangeFacetCutter create( + MultiLongValuesSource longValuesSource, + LongValuesSource singleLongValuesSource, + LongRange[] longRanges) { + if (areOverlappingRanges(longRanges)) { + return new OverlappingLongRangeFacetCutter( + longValuesSource, singleLongValuesSource, longRanges); + } + return new NonOverlappingLongRangeFacetCutter( + longValuesSource, singleLongValuesSource, longRanges); + } + + public static LongRangeFacetCutter create( + MultiLongValuesSource longValuesSource, LongRange[] longRanges) { + return create(longValuesSource, null, longRanges); + } + + // caller handles conversion of Doubles and DoubleRange to Long and LongRange + // ranges need not be sorted + LongRangeFacetCutter( + MultiLongValuesSource longValuesSource, + LongValuesSource singleLongValuesSource, + LongRange[] longRanges) { + super(); + 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; + ; Review Comment: Another lurker. ########## lucene/core/src/java/org/apache/lucene/search/DoubleValuesSource.java: ########## @@ -115,6 +116,69 @@ public final LongValuesSource toLongValuesSource() { return new LongDoubleValuesSource(this); } + public final LongValuesSource toSortableLongDoubleValuesSource() { + return new SortableLongDoubleValuesSource(this); + } + + private static class SortableLongDoubleValuesSource extends LongValuesSource { + + private final DoubleValuesSource inner; + + private SortableLongDoubleValuesSource(DoubleValuesSource inner) { + this.inner = inner; + } + + @Override + public LongValues getValues(LeafReaderContext ctx, DoubleValues scores) throws IOException { + DoubleValues in = inner.getValues(ctx, scores); + + return new LongValues() { + @Override + public long longValue() throws IOException { + return NumericUtils.doubleToSortableLong(in.doubleValue()); + } + + @Override + public boolean advanceExact(int doc) throws IOException { + return in.advanceExact(doc); + } + }; + } + + @Override + public boolean needsScores() { + return inner.needsScores(); + } + + @Override + public int hashCode() { + return Objects.hash(inner); Review Comment: I think `inner.hashCode()` is cleaner? And if we require `inner` to be non-null (above comment) that addresses the `null` handling difference. I think `Objects.hash` is helpful when you have more than one thing needing hash mixing. -- 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