Re: [PR] WIP - Add minimum number of segments to TieredMergePolicy [lucene]

2024-06-04 Thread via GitHub


jpountz commented on code in PR #13430:
URL: https://github.com/apache/lucene/pull/13430#discussion_r1625497128


##
lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java:
##
@@ -522,21 +550,28 @@ private MergeSpecification doFindMerges(
 final List candidate = new ArrayList<>();

Review Comment:
   Merging is only necessary if there are more segments than necessary or more 
deletes than necessary. So this condition just stops looking for more merges 
when none of these two conditions are met?



##
lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java:
##
@@ -522,21 +550,28 @@ private MergeSpecification doFindMerges(
 final List candidate = new ArrayList<>();
 boolean hitTooLarge = false;
 long bytesThisMerge = 0;
+long docCountThisMerge = 0;
 for (int idx = startIdx;
 idx < sortedEligible.size()
 && candidate.size() < mergeFactor
-&& bytesThisMerge < maxMergedSegmentBytes;
+&& bytesThisMerge < maxMergedSegmentBytes
+&& docCountThisMerge < allowedDocCount;
 idx++) {
   final SegmentSizeAndDocs segSizeDocs = sortedEligible.get(idx);
   final long segBytes = segSizeDocs.sizeInBytes;
-
+  int segDocCount = segSizeDocs.maxDoc - segSizeDocs.delCount;
+  if (docCountThisMerge + segDocCount > allowedDocCount) {
+// We don't want to merge segments that will produce more 
documents than allowedDocCount
+continue;

Review Comment:
   I think we should `break` here, not `continue`. `continue` allows producing 
merges of segments whose sizes are not adjacent, I don't think we should allow 
this for the doc count condition, as this potentially makes merging run in 
quadratic time.



##
lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java:
##
@@ -77,20 +77,23 @@ protected void assertSegmentInfos(MergePolicy policy, 
SegmentInfos infos) throws
 long levelSizeBytes = Math.max(minSegmentBytes, (long) 
(tmp.getFloorSegmentMB() * 1024 * 1024));
 long bytesLeft = totalBytes;
 double allowedSegCount = 0;
+final int maxNumSegmentsOnHighestTier =
+(int) Math.max(tmp.getSegmentsPerTier(), 
tmp.getTargetSearchConcurrency());
 // below we make the assumption that segments that reached the max segment
 // size divided by 2 don't need merging anymore
 int mergeFactor = (int) Math.min(tmp.getSegmentsPerTier(), 
tmp.getMaxMergeAtOnce());
 while (true) {
   final double segCountLevel = bytesLeft / (double) levelSizeBytes;
-  if (segCountLevel < tmp.getSegmentsPerTier() || levelSizeBytes >= 
maxMergedSegmentBytes / 2) {
+  if (segCountLevel < maxNumSegmentsOnHighestTier

Review Comment:
   Should it allow `maxNumSegmentsOnHighestTier` for consistency with the code 
under `main`? Though I don't expect it to make a big difference since 
`segCountLevel` is a double.
   
   ```suggestion
 if (segCountLevel <= maxNumSegmentsOnHighestTier
   ```



##
lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java:
##
@@ -522,21 +550,28 @@ private MergeSpecification doFindMerges(
 final List candidate = new ArrayList<>();
 boolean hitTooLarge = false;
 long bytesThisMerge = 0;
+long docCountThisMerge = 0;
 for (int idx = startIdx;
 idx < sortedEligible.size()
 && candidate.size() < mergeFactor
-&& bytesThisMerge < maxMergedSegmentBytes;
+&& bytesThisMerge < maxMergedSegmentBytes
+&& docCountThisMerge < allowedDocCount;
 idx++) {
   final SegmentSizeAndDocs segSizeDocs = sortedEligible.get(idx);
   final long segBytes = segSizeDocs.sizeInBytes;
-
+  int segDocCount = segSizeDocs.maxDoc - segSizeDocs.delCount;
+  if (docCountThisMerge + segDocCount > allowedDocCount) {
+// We don't want to merge segments that will produce more 
documents than allowedDocCount
+continue;

Review Comment:
   We probably also should produce a singleton merge here in case 
`candidate.isEmpty()` (see what we're doing for too large segments below). Most 
likely this suggested merge will not be selected because it will have a low 
score, though it could end up selected if it reclaims lots of deletes and we 
don't have better merges.



##
lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java:
##
@@ -916,14 +951,13 @@ public MergeSpecification 
findForcedDeletesMerges(SegmentInfos infos, MergeConte
 final Set merging = mergeContext.getMergingSegments();
 
 boolean haveWork = false;
+int totalDelCount = 0;
 for (SegmentCommitInfo info : infos) {
   int delCount = mergeContext.numDeletesToMerge(info);
   assert assertDelCount(delCount, info);
+  totalDelCount += delCount;

[PR] Sparse index: optional skip list on top of doc values [lucene]

2024-06-04 Thread via GitHub


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

Speaking to Adrien about how a sparse index would look like in lucene, he 
suggested that the sparse indexing does not need to be a new format bit an 
additional responsibility if `DocValuesFormat`. 
   
   The idea is to add an option to add a [skip 
list](https://en.wikipedia.org/wiki/Skip_list) on top of doc values and to 
expose it via the `DocValuesSkipper` abstraction, which has an API that is 
similar to what we're doing for impacts today. This provides a very simple 
index which can be very efficient when the index is sorted and the field 
belongs to the index sorting.
   
   In order to implement it, we added a new flag in `FieldType.java` that 
configures whether to create a "skip index" for doc values. This flag is only 
allowed to be set on doc values of type NUMERIC, SORTED_NUMERIC, SORTED and 
SORTED_SET. Attempting to index other type of doc values with the flag set 
results on an exception.
   
   This flag needs to be persisted on the `FieldInfosFormat`. This does not 
require a format change as we have some unused bit flags  in 
`Lucene94FieldInfosFormat` that we can use.
   
   We have changed the `DocValuesFormat` to generate the "skip index" whenever 
the flag is set. For this first implementation we went to the most basic 
implementation which consist in a skip list with just one level. In this level 
we collect the documents statistics every 4096 documents and we write them into 
the index. This basic structure already provides interesting numbers. I 
discussed with Adrien that as a follow up we should introduce more levels to 
the skip list and optimise the index for low cardinality fields.
   
   In order to index a field with a skip list, we added static methods to the 
doc values field, for example NumericDocValuesField#indexedField(String name, 
long value) which will generated the right FieldType. In order to query it, you 
can use the existing  NumericDocValuesField#newSlowRangeQuery(String field, 
long lowerValue, long upperValue). The generated query will use the skip index 
if exists by using the `DocValuesRangeIterator`. 
   
   Finally, here are some number I got using the geonames data set from lucene 
util.  
   
   The first test index the field called `modified` and adds the field as the 
primary sort of the index. 
   
   ```
Index LongField query LongField#newRangeQuery
   
 INDEX TIME: 42.604 sec
 INDEX DOCS: 12347608 documents
 INDEX SIZE: 12.402745246887207 MB
 QUERY TIME: 1157.7 ms
 QUERY DOCS: 6243379080 documents
   
   Index LongField query IndexSortSortedNumericDocValuesRangeQuery
   
 INDEX TIME: 42.562 sec
 INDEX DOCS: 12347608 documents
 INDEX SIZE: 12.402745246887207 MB
 QUERY TIME: 662.6 ms
 QUERY DOCS: 6243379080 documents
   
   Index Doc values skipping query SortedNumericDocValuesField#newSlowRangeQuery
   
 INDEX TIME: 38.927 sec
 INDEX DOCS: 12347608 documents
 INDEX SIZE: 11.800291061401367 MB
 QUERY TIME: 1072.5 ms
 QUERY DOCS: 6243379080 documents
   ```
   
   This basic implementation is already faster that querying using the bkd 
tree. The IndexSortSortedNumericDocValuesRangeQuery is faster as it contains 
many optimisations but my expectation is that we can make this index as fast if 
not faster than this implementation.
   
   The second test, we are indexing two fields and sorting the index using 
them; the countryCode as primary sort and the modified field as secondary sort. 
Then we execute the range queries on the modified field:
   
   ```
   Index KeywordField, LongField query LongField#newRangeQuery
   
 INDEX TIME: 50.486 sec
 INDEX DOCS: 12347608 documents
 INDEX SIZE: 24.378992080688477 MB
 QUERY TIME: 1273.0 ms
 QUERY DOCS: 6243379080 documents
   
   Index KeywordField, LongField query 
SortedNumericDocValuesField#newSlowRangeQuery
   
 INDEX TIME: 50.486 sec
 INDEX DOCS: 12347608 documents
 INDEX SIZE: 24.378992080688477 MB
 QUERY TIME: 13392.6 ms
 QUERY DOCS: 6243379080 documents
   
Index Doc values skipping for both, query  
SortedNumericDocValuesField#newSlowRangeQuery
 INDEX TIME: 44.127 sec
 INDEX DOCS: 12347608 documents
 INDEX SIZE: 16.09447193145752 MB
 QUERY TIME: 2975.0 ms
 QUERY DOCS: 6243379080 documents
   ```
   In this case the query is slower than the BKD tree but still much faster 
than the brute approach. The advantage of the new query is that it does not 
need to build the big bitset that we might need to build with the BKD tree.
   
   relates https://github.com/apache/lucene/issues/11432


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

Re: [PR] Avoid SegmentTermsEnumFrame reload block. [lucene]

2024-06-04 Thread via GitHub


vsop-479 commented on code in PR #13253:
URL: https://github.com/apache/lucene/pull/13253#discussion_r1625269547


##
lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnum.java:
##
@@ -434,8 +436,29 @@ public boolean seekExact(BytesRef target) throws 
IOException {
 //   System.out.println("  target is before current (shares 
prefixLen=" + targetUpto + ");
 // rewind frame ord=" + lastFrame.ord);
 // }
+
+// We got lastFrame by comparing target and term, and target less than 
last seeked term in
+// currentFrame. If lastFrame's fp is same with currentFrame's fp, we 
can reduce entCount to
+// nextEnt.
+boolean currentIsLast = currentFrame.fp == lastFrame.fp;
 currentFrame = lastFrame;
-currentFrame.rewind();
+
+// Only rewindWithoutReload for non-floor block or first floor block.
+// TODO: We need currentFrame's first entry to judge whether we can 
rewindWithoutReload for
+// non-first floor blocks.
+if (currentFrame.fp != currentFrame.fpOrig
+|| currentFrame.entCount == 0
+|| currentFrame.nextEnt == -1) {
+  currentFrame.rewind();
+} else {
+  // Prepare to reduce entCount.
+  if (currentIsLast && currentFrame.isLeafBlock) {

Review Comment:
   Maybe I was not clear. I mean we can `rewindWithoutReload` when 
`currentFrame.fp == currentFrame.fpOrig`, base on this,  if we finally seek the 
same block with last `term`, roll back `entCount` temporarily.



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



Re: [I] Instrument IndexOrDocValuesQuery to report on its decisions [lucene]

2024-06-04 Thread via GitHub


jpountz commented on issue #13442:
URL: https://github.com/apache/lucene/issues/13442#issuecomment-2147633407

   > A general framework on IndexSearcher sounds nice, but it's hard to 
generalize with just this one use case?
   
   Can it be something like IndexWriter's InfoStream, but for search? Or do we 
need/want something more structured?
   
   > Can we think of other queries/collectors that might also benefit from this?
   
   Some examples that come to mind:
- Some users may want to know what fields are used for searching or 
filtering in general, e.g. what percentage of queries are filtered by category? 
by brand? by price?
- What clause(s) leads query evaluation.
- Number of hits that get evaluated vs. cost. (ie. how much is dynamic 
pruning helping). Or maybe this one is too heavy and belongs to the query 
profiler.
- What scorer is used, e.g. BS1 vs. BS2.


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



Re: [I] Support for criteria based DWPT selection inside DocumentWriter [lucene]

2024-06-04 Thread via GitHub


jpountz commented on issue #13387:
URL: https://github.com/apache/lucene/issues/13387#issuecomment-2147657867

   > do we have such a class already (that would distinguish the tenants via 
filename prefix or so)? That's a nice idea all by itself (separate from this 
use case) -- maybe open a spinoff to explore that?
   
   I don't think we do. +1 to exploring this separately. I like that we then 
wouldn't need to tune the merge policy because it would naturally only see 
segments that belong to its group.
   
   > You would also need a clean-ish way to manage a single total allowed RAM 
bytes across the N IndexWriters? I think IndexWriter's flushing policy or RAM 
accounting was already generalized to allow for this use case, but I don't 
remember the details.
   
   Right, `IndexWriter#flushNextBuffer()` and `IndexWriter#ramBytesUsed()` 
allow building this sort of thing on top of Lucene. It would be nice if Lucene 
provided more ready-to-use utilities around this.
   
   > Searching across the N separate shards as if they were a single index is 
also possible via MultiReader, though, I'm not sure how well intra-query 
concurrency works -- maybe it works just fine because the search-time 
leaves/slices are all union'd across the N shards?
   
   Indeed, I'd expect it to work just fine.


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



[PR] Give a hint to `IndexInput` about slices that have a forward-only access pattern. [lucene]

2024-06-04 Thread via GitHub


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

   This introduces a new API that allows directories to optimize access to 
`IndexInput`s that have a forward-only access pattern by reading ahead of the 
current position. It would be applicable to:
- Postings lists,
- Doc values,
- Norms,
- Points.
   
   Relates #13179
   
   Opening as a draft PR for now to get opinions on the approach.


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



Re: [I] Improve Lucene's I/O concurrency [lucene]

2024-06-04 Thread via GitHub


jpountz commented on issue #13179:
URL: https://github.com/apache/lucene/issues/13179#issuecomment-2147824728

   @sohami I gave a try at a possible approach at #13450 in case you're curious.


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



Re: [PR] Removed Scorer#getWeight [lucene]

2024-06-04 Thread via GitHub


jpountz commented on code in PR #13440:
URL: https://github.com/apache/lucene/pull/13440#discussion_r1622334950


##
lucene/core/src/java/org/apache/lucene/search/Scorer.java:
##
@@ -26,27 +25,7 @@
  * increasing order of doc id.
  */
 public abstract class Scorer extends Scorable {
-
-  /** the Scorer's parent Weight */
-  protected final Weight weight;
-
-  /**
-   * Constructs a Scorer
-   *
-   * @param weight The scorers Weight.
-   */
-  protected Scorer(Weight weight) {
-this.weight = Objects.requireNonNull(weight);
-  }
-
-  /**
-   * returns parent Weight
-   *
-   * @lucene.experimental
-   */
-  public Weight getWeight() {
-return weight;
-  }
+  protected Scorer() {}

Review Comment:
   I'm almost surprised that the build didn't fail due to missing javadocs, I 
thought we had such a check (which also required us to define no-arg 
constructors explicitly). Can you add javadocs?



##
lucene/CHANGES.txt:
##
@@ -458,7 +460,8 @@ API Changes
   IndexSearcher#search(Query, CollectorManager) for TopFieldCollectorManager
   and TopScoreDocCollectorManager. (Zach Chen, Adrien Grand, Michael 
McCandless, Greg Miller, Luca Cavanna)
 
-* GITHUB#12854: Mark DrillSideways#createDrillDownFacetsCollector as 
@Deprecated. (Greg Miller)
+* GITHUB#12854: Mark DrillSideways#createDrillDownFacetsCollector as @
+Deprecated. (Greg Miller)

Review Comment:
   Undo the new line?



##
lucene/core/src/java/org/apache/lucene/search/Scorer.java:
##
@@ -39,15 +39,6 @@ protected Scorer(Weight weight) {
 this.weight = Objects.requireNonNull(weight);

Review Comment:
   Can you also remove the `Weight` parameter from the constructor? This is the 
most annoying bit to me.



##
lucene/core/src/java/org/apache/lucene/search/Scorer.java:
##
@@ -26,27 +25,7 @@
  * increasing order of doc id.
  */
 public abstract class Scorer extends Scorable {
-
-  /** the Scorer's parent Weight */
-  protected final Weight weight;
-
-  /**
-   * Constructs a Scorer
-   *
-   * @param weight The scorers Weight.
-   */
-  protected Scorer(Weight weight) {
-this.weight = Objects.requireNonNull(weight);
-  }
-
-  /**
-   * returns parent Weight
-   *
-   * @lucene.experimental
-   */
-  public Weight getWeight() {
-return weight;
-  }
+  protected Scorer() {}

Review Comment:
   I'm almost surprised that the build didn't fail due to missing javadocs, I 
thought we had such a check (which also required us to define no-arg 
constructors explicitly). Can you add javadocs?



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

To unsubscribe, e-mail: issues-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



Re: [PR] Removed Scorer#getWeight [lucene]

2024-06-04 Thread via GitHub


jpountz commented on code in PR #13440:
URL: https://github.com/apache/lucene/pull/13440#discussion_r1620860434


##
lucene/core/src/java/org/apache/lucene/search/Scorer.java:
##
@@ -39,15 +39,6 @@ protected Scorer(Weight weight) {
 this.weight = Objects.requireNonNull(weight);

Review Comment:
   Can you also remove the `Weight` parameter from the constructor? This is the 
most annoying bit to me.



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



Re: [PR] Rewrite newSlowRangeQuery to MatchNoDocsQuery when upper > lower [lucene]

2024-06-04 Thread via GitHub


jpountz merged PR #13425:
URL: https://github.com/apache/lucene/pull/13425


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



Re: [PR] WIP - Add minimum number of segments to TieredMergePolicy [lucene]

2024-06-04 Thread via GitHub


carlosdelest commented on code in PR #13430:
URL: https://github.com/apache/lucene/pull/13430#discussion_r1626064947


##
lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java:
##
@@ -522,21 +550,28 @@ private MergeSpecification doFindMerges(
 final List candidate = new ArrayList<>();
 boolean hitTooLarge = false;
 long bytesThisMerge = 0;
+long docCountThisMerge = 0;
 for (int idx = startIdx;
 idx < sortedEligible.size()
 && candidate.size() < mergeFactor
-&& bytesThisMerge < maxMergedSegmentBytes;
+&& bytesThisMerge < maxMergedSegmentBytes
+&& docCountThisMerge < allowedDocCount;
 idx++) {
   final SegmentSizeAndDocs segSizeDocs = sortedEligible.get(idx);
   final long segBytes = segSizeDocs.sizeInBytes;
-
+  int segDocCount = segSizeDocs.maxDoc - segSizeDocs.delCount;
+  if (docCountThisMerge + segDocCount > allowedDocCount) {
+// We don't want to merge segments that will produce more 
documents than allowedDocCount
+continue;

Review Comment:
   Makes sense, thanks!



##
lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java:
##
@@ -77,20 +77,23 @@ protected void assertSegmentInfos(MergePolicy policy, 
SegmentInfos infos) throws
 long levelSizeBytes = Math.max(minSegmentBytes, (long) 
(tmp.getFloorSegmentMB() * 1024 * 1024));
 long bytesLeft = totalBytes;
 double allowedSegCount = 0;
+final int maxNumSegmentsOnHighestTier =
+(int) Math.max(tmp.getSegmentsPerTier(), 
tmp.getTargetSearchConcurrency());
 // below we make the assumption that segments that reached the max segment
 // size divided by 2 don't need merging anymore
 int mergeFactor = (int) Math.min(tmp.getSegmentsPerTier(), 
tmp.getMaxMergeAtOnce());
 while (true) {
   final double segCountLevel = bytesLeft / (double) levelSizeBytes;
-  if (segCountLevel < tmp.getSegmentsPerTier() || levelSizeBytes >= 
maxMergedSegmentBytes / 2) {
+  if (segCountLevel < maxNumSegmentsOnHighestTier

Review Comment:
   Good catch - updating



##
lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java:
##
@@ -522,21 +550,28 @@ private MergeSpecification doFindMerges(
 final List candidate = new ArrayList<>();

Review Comment:
   > Merging is only necessary if there are more segments than necessary or 
more deletes than necessary.
   
   The problem is that sortedEligible has filtered out the segments that cannot 
be merged (like those that are bigger than max segment size), so this condition 
will skip trying to merge even if we have more segments than allowed. 
   
   So for example we may have 17 segments total, from which we can just have 12 
sortedEligibles, and have 15 allowed segment count. We would never try to merge 
the eligible segments as they're less than the number of allowed segment counts.
   
   Should we take into account the total number of segments vs the total 
eligibles for this condition?



##
lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java:
##
@@ -111,11 +114,29 @@ protected void assertSegmentInfos(MergePolicy policy, 
SegmentInfos infos) throws
   }
 }
 
+// There can be more segments if we can't merge docs - they are balanced 
between segments
+int maxDocsPerSegment = tmp.getMaxAllowedDocs(infos.totalMaxDoc());

Review Comment:
   Yes, we should. Thanks!



##
lucene/core/src/test/org/apache/lucene/index/TestTieredMergePolicy.java:
##
@@ -111,11 +114,29 @@ protected void assertSegmentInfos(MergePolicy policy, 
SegmentInfos infos) throws
   }
 }
 
+// There can be more segments if we can't merge docs - they are balanced 
between segments
+int maxDocsPerSegment = tmp.getMaxAllowedDocs(infos.totalMaxDoc());
+List segmentDocs =
+infos.asList().stream()
+.map(info -> info.info.maxDoc() - info.getDelCount())
+.sorted()
+.toList();
+int numEligible = 0;
+int currentSum = 0;
+for (int i = 0; i < segmentDocs.size(); i++) {
+  currentSum += segmentDocs.get(i);
+  if (currentSum > maxDocsPerSegment) {
+break;
+  }
+  numEligible++;
+}
+boolean eligibleDocsMerge = numEligible > 1;

Review Comment:
   Correct, applied the change.



##
lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java:
##
@@ -916,14 +951,13 @@ public MergeSpecification 
findForcedDeletesMerges(SegmentInfos infos, MergeConte
 final Set merging = mergeContext.getMergingSegments();
 
 boolean haveWork = false;
+int totalDelCount = 0;
 for (SegmentCommitInfo info : infos) {
   int delCount = mergeContext.numDeletesToMerge(info);
   assert assertDelCount(delCount, info);
+  totalDelCount += delCount;
   double pctDe

Re: [PR] Give a hint to `IndexInput` about slices that have a forward-only access pattern. [lucene]

2024-06-04 Thread via GitHub


rmuir commented on code in PR #13450:
URL: https://github.com/apache/lucene/pull/13450#discussion_r1626474331


##
lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInput.java:
##
@@ -370,6 +370,16 @@ public void prefetch(long offset, long length) throws 
IOException {
 }
   }
 
+  @Override
+  public void readAhead(long offset, long length) throws IOException {
+// Start loading the first bytes in the background
+if (length != 0) {
+  prefetch(offset, 1);
+}
+// TODO: Is there a hint we can give to the OS to let it optimize for our 
forward-only access
+// pattern in the given range?
+  }

Review Comment:
   `MADV_SEQUENTIAL`



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



Re: [I] AnalyzingSuggester exception because of length restriction: java.lang.IllegalArgumentException: len must be <= 32767; got 38751 [LUCENE-6012] [lucene]

2024-06-04 Thread via GitHub


dmaziuk commented on issue #7074:
URL: https://github.com/apache/lucene/issues/7074#issuecomment-2148222112

   +1: trying to set up the suggester, got `len must be <= 32767; got 38822`
   
   How am I supposed to guarantee that the field I'm pulling from external 
sources? Use the `string` field instead of `text`? Will that work a s target 
for `copyField` where the sources may add up to more than 32766?
   


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



Re: [PR] Sparse index: optional skip list on top of doc values [lucene]

2024-06-04 Thread via GitHub


ChrisHegarty commented on code in PR #13449:
URL: https://github.com/apache/lucene/pull/13449#discussion_r1626526797


##
lucene/core/src/java/org/apache/lucene/codecs/lucene90/Lucene90DocValuesProducer.java:
##
@@ -1690,4 +1722,78 @@ long getLongValue(long index) throws IOException {
   return mul * values.get(index & mask) + delta;
 }
   }
+
+  @Override
+  public DocValuesSkipper getSkipper(FieldInfo field) throws IOException {
+final DocValuesSkipperEntry entry = skippers.get(field.name);
+
+final IndexInput input = data.slice("doc value skipper", entry.offset, 
entry.length);
+return new DocValuesSkipper() {
+  int minDocID = -1;
+  int maxDocID = -1;
+  long minValue, maxValue;
+  int docCount;
+
+  @Override
+  public void advance(int target) throws IOException {
+if (target > entry.maxDocId) {
+  minDocID = DocIdSetIterator.NO_MORE_DOCS;
+  maxDocID = DocIdSetIterator.NO_MORE_DOCS;
+} else {
+  do {
+minDocID = input.readInt();
+maxDocID = input.readInt();
+minValue = input.readLong();
+maxValue = input.readLong();
+docCount = input.readInt();
+  } while (target > maxDocID);

Review Comment:
   I'm a little confused by this - it looks wrong to me.  Since something as 
simple as:
   ```
 skipper.advance(doc);
 skipper.advance(doc);
   ```
   will have a side-effect, when it should not, right?



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

To unsubscribe, e-mail: issues-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



Re: [PR] Sparse index: optional skip list on top of doc values [lucene]

2024-06-04 Thread via GitHub


ChrisHegarty commented on code in PR #13449:
URL: https://github.com/apache/lucene/pull/13449#discussion_r1626571087


##
lucene/core/src/java/org/apache/lucene/index/DocValuesSkipper.java:
##
@@ -0,0 +1,95 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+import org.apache.lucene.search.DocIdSetIterator;
+
+/** Skipper for {@link DocValues}. */
+public abstract class DocValuesSkipper {

Review Comment:
   I struggled with this interface initially, so I'll propose a few javadoc 
comments which I think should help.
   
   ```
* A skipper has a position that can only be {@link #advance(int) 
advanced}. A skippers position,
* along with a {@code level}, determines the interval at which the skipper 
is currently positioned.
```



##
lucene/core/src/java/org/apache/lucene/index/DocValuesSkipper.java:
##
@@ -0,0 +1,95 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+import org.apache.lucene.search.DocIdSetIterator;
+
+/** Skipper for {@link DocValues}. */
+public abstract class DocValuesSkipper {
+
+  /**
+   * Advance this skipper so that all levels contain the next document on or 
after {@code target}.
+   *
+   * NOTE: The behavior is undefined if {@code target} is less than 
or equal to {@code
+   * maxDocID(0)}.
+   *
+   * NOTE: {@code minDocID(0)} may return a doc ID that is greater 
than {@code target} if
+   * the target document doesn't have a value.
+   */
+  public abstract void advance(int target) throws IOException;
+
+  /** Return the number of levels. This number may change when moving to a 
different interval. */
+  public abstract int numLevels();
+
+  /**
+   * Return the minimum doc ID on the given level, inclusive. This returns 
{@code -1} if {@link
+   * #advance(int)} has not been called yet and {@link 
DocIdSetIterator#NO_MORE_DOCS} if the
+   * iterator is exhausted. This method is non-increasing when {@code level} 
increases. Said
+   * otherwise {@code minDocID(level+1) <= minDocId(level)}.
+   */
+  public abstract int minDocID(int level);
+
+  /**
+   * Return the maximum doc ID on the given level, inclusive. This returns 
{@code -1} if {@link
+   * #advance(int)} has not been called yet and {@link 
DocIdSetIterator#NO_MORE_DOCS} if the
+   * iterator is exhausted. This method is non-decreasing when {@code level} 
decreases. Said
+   * otherwise {@code maxDocID(level+1) >= maxDocId(level)}.
+   */
+  public abstract int maxDocID(int level);
+
+  /**
+   * Return the minimum value at the given level, inclusive.
+   *
+   * NOTE: It is only guaranteed that values in this interval are 
greater than or equal
+   * the returned value. There is no guarantee that one document actually has 
this value.
+   */
+  public abstract long minValue(int level);
+
+  /**
+   * Return the maximum value at the given level, inclusive.

Review Comment:
   same - ".. of the internal .. "



##
lucene/core/src/java/org/apache/lucene/index/DocValuesSkipper.java:
##
@@ -0,0 +1,95 @@
+/*
+ * 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 n

Re: [PR] Introduces efSearch as a separate parameter in KNN{Byte:Float}VectorQuery [lucene]

2024-06-04 Thread via GitHub


benwtrent commented on PR #13407:
URL: https://github.com/apache/lucene/pull/13407#issuecomment-2148390886

   > For HNSW efSearch is a core parameters during search time. This is 
convenient for users to not have to have the logic to strip off top k values on 
their end.
   
   I understand, but this is trivial to do yourself by collecting and rewriting 
the query directly.
   
   > Why not Lucene have the support for something that is a core parameter, 
when its cheap enough to have it so users have the option to choose how they 
want to use it?
   
   While it is cheap, it is a further coupling of kNN to HNSW. Now the user 
interface itself when querying assumes "HNSW-esque" things. Instead of a simple 
"Give me k values" its now "give me k values with this particular HNSW 
parameter". 
   
   There has been talk of adding a "flat" codec to Lucene, that simply takes 
advantage of quantization and stores the vectors not in the HNSW graph. In that 
instance, what would `efSearch` mean?


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



Re: [I] Improve Lucene's I/O concurrency [lucene]

2024-06-04 Thread via GitHub


sohami commented on issue #13179:
URL: https://github.com/apache/lucene/issues/13179#issuecomment-2148405028

   > @sohami I gave a try at a possible approach at #13450 in case you're 
curious.
   
   @jpountz Thanks for sharing this. Originally I was thinking the prefetch 
optimization only in collect phase but I am trying to understand if it can be 
used in iterators side of things as well. To understand better I am looking 
into `SortedNumericDocValuesRangeQuery` test to understand the flow when 
different iterators are involved.  
   
   So far my general understanding is all the scoring and collection of docs 
via Collectors happens in the method `DefaultBulkScorer::Score`. The lead 
`scorerIterator` in that could either be a standalone iterator or wrapper on 
multiple iterators or an `approximation` iterator when `TwoPhaseIterator` is 
non-null. These are then passed down to `scoreAll` or `scoreRange` (ignoring 
the `competitiveIterator` for now). In either of `scoreAll` or `scoreRange` we 
iterate over the lead `scorerIterator` to get the matching docs and then check 
if the doc matches the `TwoPhaseIterator` or not to make it eligible for 
collection via collectors. So I see following flows/cases: a) When only lead 
`scorerIterator` is present, b) When both lead `scorerIterator` and 
`TwoPhaseIterator` is present, c) the collect phase which happens over doc that 
scorers have found.
   
   Based on my above understanding, I am thinking below and would love your 
feedback
   
   1.  For case (a), when only single iterator is involved the `readAhead` 
mechanism can be useful. This is considering a single iterator will not know 
what next match is until it goes to the next doc.
   
   2. For case (b), we can potentially do combination of `readAhead` and 
`prefetch`. We can use `readAhead` on lead iterator and then buffer some of the 
matching docs from this lead iterator. Then before evaluating if these docs 
matches `TwoPhaseIterator` or not, we can perform prefetch on these buffered 
docs (via some `prepareMatches` mechanism on `TwoPhaseIterator`). Here we know 
which all docs will be used for evaluating matches on `TwoPhaseIterator`, so we 
should be able to prefetch data for those docs. Would like to understand more 
on your earlier feedback on this, as my understanding is collection will come 
afterwards.
   > maybe we buffer the next few doc IDs from the first-phase scorer and 
prefetch those
   
   >> FWIW this would break a few things, e.g. we have collectors that only 
compute the score when needed (e.g. when sorting by field then score). But if 
we need to buffer docs up-front, then we don't know at this point in time if 
scores are going to be needed or not, so we need to score more docs. Maybe it's 
still the right trade-off, I'm mostly pointing out that this would be a bigger 
trade-off than what we've done for prefetching until now.
   
   3. Before calling collect phase on collectors, we can first buffer up the 
matching docs. Ask collectors to trigger optional `prefetch` of the docs which 
will be passed to it for collection. These docs are the ones which was produced 
by scorers with or without TwoPhaseIterator in the mix.
   
   I think for scenarios like 2 and 3 above where we know exact doc matches, 
performing prefetch could be useful vs readAhead.


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



Re: [PR] Introduces efSearch as a separate parameter in KNN{Byte:Float}VectorQuery [lucene]

2024-06-04 Thread via GitHub


navneet1v commented on PR #13407:
URL: https://github.com/apache/lucene/pull/13407#issuecomment-2148532097

   > There has been talk of adding a "flat" codec to Lucene, that simply takes 
advantage of quantization and stores the vectors not in the HNSW graph. In that 
instance, what would efSearch mean?
   
   @benwtrent I really excited to see what are developing on the `flat` codec 
part. But coming back to `ef_search` as a parameter, I feel this is an 
essential parameter for HNSW. So, if we can abstract this and any other 
parameter that can come in future for any algorithm in a class 
`SearchParameters` or `VectorSearchParameters` it will solve this use case and 
also gives the flexibility to add new algorithm related search parameters later 
on. 


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



Re: [PR] Fix IndexOutOfBoundsException thrown in DefaultPassageFormatter by unordered matches [lucene]

2024-06-04 Thread via GitHub


github-actions[bot] commented on PR #13315:
URL: https://github.com/apache/lucene/pull/13315#issuecomment-2148615171

   This PR has not had activity in the past 2 weeks, labeling it as stale. If 
the PR is waiting for review, notify the d...@lucene.apache.org list. Thank you 
for your contribution!


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



Re: [PR] Add BitVectors format and make flat vectors format easier to extend [lucene]

2024-06-04 Thread via GitHub


navneet1v commented on PR #13288:
URL: https://github.com/apache/lucene/pull/13288#issuecomment-2148642670

   @benwtrent 
   
   I am little confused here. I am still looking for an ans of this question: 
`Does this mean now Lucene supports BitVectorsFormat officially? Or it was more 
a prototype and not intended for production use?` 
   
   Another place where I don't have clarity is: what is the point of 
VectorSimilarity functions in case of bitvectors format. I can set a 
`MAX_INNER_PRODUCT` for bits vectors but the codec will use Hamming distance 
for similarity calculation. So it means getting setting vector similarity from 
a field is not the source truth for what vector similarity function to be used. 
Hence the implementations should come up with other ways to know what is the 
vector similarity function.
   


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



Re: [PR] Give a hint to `IndexInput` about slices that have a forward-only access pattern. [lucene]

2024-06-04 Thread via GitHub


rmuir commented on code in PR #13450:
URL: https://github.com/apache/lucene/pull/13450#discussion_r1626828513


##
lucene/core/src/java21/org/apache/lucene/store/MemorySegmentIndexInput.java:
##
@@ -370,6 +370,16 @@ public void prefetch(long offset, long length) throws 
IOException {
 }
   }
 
+  @Override
+  public void readAhead(long offset, long length) throws IOException {
+// Start loading the first bytes in the background
+if (length != 0) {
+  prefetch(offset, 1);
+}
+// TODO: Is there a hint we can give to the OS to let it optimize for our 
forward-only access
+// pattern in the given range?
+  }

Review Comment:
   or even better, `MADV_NORMAL` for the range, which is less aggressive.



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



[I] The Closeable interface of CloseableThreadLocal seems redundent [lucene]

2024-06-04 Thread via GitHub


Daniel-Chang-T opened a new issue, #13451:
URL: https://github.com/apache/lucene/issues/13451

   ### Description
   
   ## Description
   While reading the source code, I noticed that the `CloseableThreadLocal` 
implementation should release the stored hard references even without invoking 
`close()`. To illustrate this, I created the following code example.
   ```
   CloseableThreadLocal ctl = new CloseableThreadLocal<>();
   Object value = new Object();
   ctl.set(value);
   WeakReference valueRef = new WeakReference<>(value);
   value = null;
   System.gc();
   assert valueRef.get() != null;
   ctl = null;
   System.gc();
   assert valueRef.get() == null;
   ```
   This approach is advantageous because it removes the necessity to maintain 
references and manually invoking close() for classes that using 
CloseableThreadLocal e.g. Analyzer.


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



[I] org.apache.lucene.index.IndexFormatTooNewException on ARM64 [lucene]

2024-06-04 Thread via GitHub


suddendust opened a new issue, #13452:
URL: https://github.com/apache/lucene/issues/13452

   ### Description
   
   I am trying to run Apache Pinot on ARM64 (Graviton). While the table 
partitions are loaded successfully on x86, I get the following exception on ARM:
   
   ```
   org.apache.lucene.index.IndexFormatTooNewException: Format version is not 
supported (resource ): 9 (needs to be between 6 and 8)
at 
org.apache.lucene.codecs.CodecUtil.checkHeaderNoMagic(CodecUtil.java:213) 
~[startree-pinot-all-1.2.0-ST.10-jar-with-dependencies.jar:1.2.0-ST.10-ed1c4f0f376a258e77fee8db54706ded71003843]
at org.apache.lucene.codecs.CodecUtil.checkHeader(CodecUtil.java:193) 
~[startree-pinot-all-1.2.0-ST.10-jar-with-dependencies.jar:1.2.0-ST.10-ed1c4f0f376a258e77fee8db54706ded71003843]
at org.apache.lucene.util.fst.FST.(FST.java:454) 
~[startree-pinot-all-1.2.0-ST.10-jar-with-dependencies.jar:1.2.0-ST.10-ed1c4f0f376a258e77fee8db54706ded71003843]
at 
org.apache.pinot.segment.local.segment.index.readers.LuceneFSTIndexReader.(LuceneFSTIndexReader.java:55)
 
~[startree-pinot-all-1.2.0-ST.10-jar-with-dependencies.jar:1.2.0-ST.10-ed1c4f0f376a258e77fee8db54706ded71003843]
at 
org.apache.pinot.segment.local.segment.index.fst.FstIndexType$ReaderFactory.createIndexReader(FstIndexType.java:200)
 
~[startree-pinot-all-1.2.0-ST.10-jar-with-dependencies.jar:1.2.0-ST.10-ed1c4f0f376a258e77fee8db54706ded71003843]
at 
org.apache.pinot.segment.local.segment.index.fst.FstIndexType$ReaderFactory.createIndexReader(FstIndexType.java:208)
 
~[startree-pinot-all-1.2.0-ST.10-jar-with-dependencies.jar:1.2.0-ST.10-ed1c4f0f376a258e77fee8db54706ded71003843]
at 
org.apache.pinot.segment.local.segment.index.fst.FstIndexType$ReaderFactory.createIndexReader(FstIndexType.java:183)
 
~[startree-pinot-all-1.2.0-ST.10-jar-with-dependencies.jar:1.2.0-ST.10-ed1c4f0f376a258e77fee8db54706ded71003843]
at 
org.apache.pinot.segment.spi.index.IndexReaderFactory$Default.createIndexReader(IndexReaderFactory.java:67)
 
~[startree-pinot-all-1.2.0-ST.10-jar-with-dependencies.jar:1.2.0-ST.10-ed1c4f0f376a258e77fee8db54706ded71003843]
at 
org.apache.pinot.segment.local.segment.index.column.PhysicalColumnIndexContainer.(PhysicalColumnIndexContainer.java:59)
 
~[startree-pinot-all-1.2.0-ST.10-jar-with-dependencies.jar:1.2.0-ST.10-ed1c4f0f376a258e77fee8db54706ded71003843]
at 
   ```
   
   While I am still in the process of debugging this, I created this issue in 
case I can get some help faster. Thanks!
   
   ### Version and environment details
   
   Linux/arm64 
   Lucene version: 9.10.0


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