[I] DV update files referenced by merge will be deleted by concurrent flush [lucene]

2024-01-16 Thread via GitHub


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

   ### Description
   
   DV update files referenced by merge will be deleted by concurrent flush.
   For example, according to the following execution sequence, the DV update 
file referenced by merge will be deleted.
   
   1. `IndexWriter#addDocument` doc1 = {"id": "1", "version": "1"}
   2. `IndexWriter#flush`, which will produce segment0 = {_0.cfe, _0.cfs, _0.si}
   3. `IndexWriter#addDocument` = {"id": "2", "version": "1"}
   4. `IndexWriter#softUpdateDocument` updates doc2 to {"id": "2", "version": 
"2"} 
   5. `IndexWriter#flush`, which will produce segment1 = {_1.cfe, _1.cfs, 
_1.si, _1_1.fnm, _1_1_Lucene80_0.dvd, _1_1_Lucene80_0.dvm}
   6. `IndexWriter#forceMerge`, merge segment0 and segment1 into segment2. The 
merge process will reference segment0 = {_0.cfe, _0.cfs, _0.si} and segment1 = 
{_1.cfe, _1.cfs, _1.si, _1_1.fnm, _1_1_Lucene80_0.dvd, _1_1_Lucene80_0.dvm}
   7. During merge, `IndexWriter#softUpdateDocument` updates doc2 to {"id": 
"2", "version": "3"}
   8. `IndexWriter#flush`,which will produce segment3 = {_3.cfe, _3.cfs, 
_3.si},and update segment1 to {_1.cfe, _1.cfs, _1.si, _1_2.fnm, 
_1_2_Lucene80_0.dvd, _1_2_Lucene80_0.dvm}
   9. The files = {_1_1.fnm, _1_1_Lucene80_0.dvd, _1_1_Lucene80_0.dvm} 
referenced by merge is deleted
   
   In the Linux environment, an open file is still readable after being 
deleted, so it does not cause file reading exception.
   However, in other environment that do not have the above guarantee, an 
exception will be caused because the file does not exist.
   
   Later I will file a PR for this. Add related tests and fix the problem. 
   
   ### Version and environment details
   
   I found the problem in releases/lucene-solr/8.7.0, and confirmed that the 
problem still exists in main branch.


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



Re: [PR] Speedup concurrent multi-segment HNWS graph search 2 [lucene]

2024-01-16 Thread via GitHub


jimczi commented on code in PR #12962:
URL: https://github.com/apache/lucene/pull/12962#discussion_r1453088407


##
lucene/core/src/java/org/apache/lucene/search/TopKnnCollector.java:
##
@@ -27,25 +29,72 @@
  */
 public final class TopKnnCollector extends AbstractKnnCollector {
 
+  // greediness of globally non-competitive search: [0,1]
+  private static final float DEFAULT_GREEDINESS = 0.9f;
   private final NeighborQueue queue;
+  private final float greediness;
+  private final FloatHeap nonCompetitiveQueue;
+  private final FloatHeap updatesQueue;
+  private final int interval = 0x3ff; // 1023
+  private final BlockingFloatHeap globalSimilarityQueue;
+  private boolean kResultsCollected = false;
+  private float cachedGlobalMinSim = Float.NEGATIVE_INFINITY;
 
   /**
* @param k the number of neighbors to collect
* @param visitLimit how many vector nodes the results are allowed to visit
*/
-  public TopKnnCollector(int k, int visitLimit) {
+  public TopKnnCollector(int k, int visitLimit, BlockingFloatHeap 
globalSimilarityQueue) {
 super(k, visitLimit);
+this.greediness = DEFAULT_GREEDINESS;
 this.queue = new NeighborQueue(k, false);
+this.globalSimilarityQueue = globalSimilarityQueue;
+
+if (globalSimilarityQueue == null) {
+  this.nonCompetitiveQueue = null;
+  this.updatesQueue = null;
+} else {
+  this.nonCompetitiveQueue = new FloatHeap(Math.max(1, Math.round((1 - 
greediness) * k)));
+  this.updatesQueue = new FloatHeap(k);
+}
   }
 
   @Override
   public boolean collect(int docId, float similarity) {
-return queue.insertWithOverflow(docId, similarity);
+boolean localSimUpdated = queue.insertWithOverflow(docId, similarity);
+boolean firstKResultsCollected = (kResultsCollected == false && 
queue.size() == k());
+if (firstKResultsCollected) {
+  kResultsCollected = true;
+}
+
+boolean globalSimUpdated = false;
+if (globalSimilarityQueue != null) {
+  updatesQueue.offer(similarity);
+  globalSimUpdated = nonCompetitiveQueue.offer(similarity);
+
+  if (kResultsCollected) {
+// as we've collected k results, we can start do periodic updates with 
the global queue
+if (firstKResultsCollected || (visitedCount & interval) == 0) {
+  cachedGlobalMinSim = 
globalSimilarityQueue.offer(updatesQueue.getHeap());

Review Comment:
   Sorry I meant if `k` is close to the interval (1024). In such case delaying 
the update to the global queue doesn't save much.



-- 
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] Fix DV update files referenced by merge will be deleted by concurrent flush [lucene]

2024-01-16 Thread via GitHub


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

   ### Description
   
   This PR aims to address issue #13015.
   A more detailed explanation of the issue and the reasoning behind the fix 
can be found in the report link above.
   
   ### Solution
   
   - Before the file merge starts, execute `IndexFileDeleter#incRef` on the 
files referenced by merge to ensure that the file will not be deleted.
   - After the file merge is completed/abort, execute `IndexFileDeleter#decRef`.
   - In `ReadersAndUpdates#writeFieldUpdates`, 
`ReadersAndUpdates#swapNewReaderWithLatestLiveDocs` should be executed after 
`SegmentCommitInfo#setDocValuesUpdatesFiles`, ensuring that `SegmentReader` 
contains a deep clone of the `SegmentCommitInfo#dvUpdatesFiles`.
   
   ### Tests
   
   - I introduced `MockAssertFileExistIndexInput` which validate file exists 
before reading it.
   - I introduced `testForceMergeDVUpdateFileWithConcurrentFlush` which fails 
on `main` branch since the DV update file not exist, but passes on this branch.
   - I added corresponding 
`testMergeDVUpdateFileOnGetReaderWithConcurrentFlush` which passes on both 
`main` and this branch. Because before merge, `IndexFileDeleter#incRef` is 
executed in `StandardDirectoryReader#open`.
   - I added corresponding `testMergeDVUpdateFileOnCommitWithConcurrentFlush` 
which passes on both `main` and this branch. Because before merge, 
`IndexFileDeleter#incRef` is executed in `IndexWriter#prepareCommitInternal`.
   
   ### Checklist
   
   - [x]  I have reviewed the guidelines for [How to 
Contribute](https://github.com/apache/lucene/blob/main/CONTRIBUTING.md) and my 
code conforms to the standards described there to the best of my ability.
   - [x]  I have given Lucene maintainers 
[access](https://docs.github.com/en/pull-requests/collaborating-with-pull-requests/working-with-forks/allowing-changes-to-a-pull-request-branch-created-from-a-fork)
 to contribute to my PR branch. (optional but recommended)
   - [x]  I have developed this patch against the `main` branch.
   - [x]  I have run `./gradlew check`.
   - [x]  I have added tests for my changes.
   


-- 
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] LUCENE-10366: Override #readVInt and #readVLong for ByteBufferDataInput to avoid the abstraction confusion of #readByte. [lucene]

2024-01-16 Thread via GitHub


gf2121 merged PR #592:
URL: https://github.com/apache/lucene/pull/592


-- 
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] DV update files referenced by merge will be deleted by concurrent flush [lucene]

2024-01-16 Thread via GitHub


guojialiang92 commented on issue #13015:
URL: https://github.com/apache/lucene/issues/13015#issuecomment-1893384217

   I saw a similar issue [11751](https://github.com/apache/lucene/issues/11751).


-- 
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] Prevent parent field from added to existing index [lucene]

2024-01-16 Thread via GitHub


s1monw merged PR #13016:
URL: https://github.com/apache/lucene/pull/13016


-- 
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] LUCENE-10366: Override #readVInt and #readVLong for ByteBufferDataInput to avoid the abstraction confusion of #readByte. [lucene]

2024-01-16 Thread via GitHub


uschindler commented on PR #592:
URL: https://github.com/apache/lucene/pull/592#issuecomment-1893411279

   Thanks!


-- 
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] Backport #12829 to 9.x [lucene]

2024-01-16 Thread via GitHub


s1monw closed pull request #13013: Backport #12829 to 9.x
URL: https://github.com/apache/lucene/pull/13013


-- 
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] Backport #12829 to 9.x [lucene]

2024-01-16 Thread via GitHub


s1monw commented on PR #13013:
URL: https://github.com/apache/lucene/pull/13013#issuecomment-1893514491

   this doesn't work as I expected. I will close this one for now


-- 
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] Modernize LineFileDocs. [lucene]

2024-01-16 Thread via GitHub


s1monw commented on PR #12929:
URL: https://github.com/apache/lucene/pull/12929#issuecomment-1893520487

   I was working on adding some new BWC tests for the parent field and it seems 
BWC index creation and testing is relying on some things that changed here. 
Just flagging this for now while I work on a fix.


-- 
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] Speedup concurrent multi-segment HNWS graph search 2 [lucene]

2024-01-16 Thread via GitHub


tveasey commented on code in PR #12962:
URL: https://github.com/apache/lucene/pull/12962#discussion_r1453309333


##
lucene/core/src/java/org/apache/lucene/search/TopKnnCollector.java:
##
@@ -27,25 +29,72 @@
  */
 public final class TopKnnCollector extends AbstractKnnCollector {
 
+  // greediness of globally non-competitive search: [0,1]
+  private static final float DEFAULT_GREEDINESS = 0.9f;
   private final NeighborQueue queue;
+  private final float greediness;
+  private final FloatHeap nonCompetitiveQueue;
+  private final FloatHeap updatesQueue;
+  private final int interval = 0x3ff; // 1023

Review Comment:
   This seems quite large to me, based on the total node visited counts in the 
Cohere results. For example, for fo = 90 we'd only refresh around twice.



##
lucene/core/src/java/org/apache/lucene/search/TopKnnCollector.java:
##
@@ -27,25 +29,72 @@
  */
 public final class TopKnnCollector extends AbstractKnnCollector {
 
+  // greediness of globally non-competitive search: [0,1]
+  private static final float DEFAULT_GREEDINESS = 0.9f;
   private final NeighborQueue queue;
+  private final float greediness;
+  private final FloatHeap nonCompetitiveQueue;
+  private final FloatHeap updatesQueue;
+  private final int interval = 0x3ff; // 1023
+  private final BlockingFloatHeap globalSimilarityQueue;
+  private boolean kResultsCollected = false;
+  private float cachedGlobalMinSim = Float.NEGATIVE_INFINITY;
 
   /**
* @param k the number of neighbors to collect
* @param visitLimit how many vector nodes the results are allowed to visit
*/
-  public TopKnnCollector(int k, int visitLimit) {
+  public TopKnnCollector(int k, int visitLimit, BlockingFloatHeap 
globalSimilarityQueue) {
 super(k, visitLimit);
+this.greediness = DEFAULT_GREEDINESS;
 this.queue = new NeighborQueue(k, false);
+this.globalSimilarityQueue = globalSimilarityQueue;
+
+if (globalSimilarityQueue == null) {
+  this.nonCompetitiveQueue = null;
+  this.updatesQueue = null;
+} else {
+  this.nonCompetitiveQueue = new FloatHeap(Math.max(1, Math.round((1 - 
greediness) * k)));
+  this.updatesQueue = new FloatHeap(k);
+}
   }
 
   @Override
   public boolean collect(int docId, float similarity) {
-return queue.insertWithOverflow(docId, similarity);
+boolean localSimUpdated = queue.insertWithOverflow(docId, similarity);
+boolean firstKResultsCollected = (kResultsCollected == false && 
queue.size() == k());
+if (firstKResultsCollected) {
+  kResultsCollected = true;
+}
+
+boolean globalSimUpdated = false;
+if (globalSimilarityQueue != null) {
+  updatesQueue.offer(similarity);
+  globalSimUpdated = nonCompetitiveQueue.offer(similarity);
+
+  if (kResultsCollected) {
+// as we've collected k results, we can start do periodic updates with 
the global queue
+if (firstKResultsCollected || (visitedCount & interval) == 0) {
+  cachedGlobalMinSim = 
globalSimilarityQueue.offer(updatesQueue.getHeap());

Review Comment:
   There is a subtlety: note that `firstKResultsCollected` is only true the 
first time we visit k nodes, so this is saying only refresh every 1024 
iterations thereafter. The refresh schedule is `k, k + 1024, k + 2048, ...`. 
(The || threw me initially.)
   
   As per my comment above, 1024 seems infrequent to me. (Of course you may 
have tested smaller values and determined this to be a good choice.) If we 
think it is risky sharing information too early, I would be inclined to share 
on the schedule `max(k, c1), max(k, c1) + c2, max(k, c1) + 2 * c2, ...` with 
`c1 > c2` and decouple the concerns.
   
   Also, there maybe issues with *using* information too early, in which case 
`minCompetitiveSimilarity` would need a check that `visitedCount > max(k, c1)` 
before it starts modifying `minSim`.
   
   In any case, I can see results being rather sensitive to these choices so if 
you haven't done a parameter exploration it might be worth trying a few choices.



-- 
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] Speedup concurrent multi-segment HNWS graph search 2 [lucene]

2024-01-16 Thread via GitHub


tveasey commented on code in PR #12962:
URL: https://github.com/apache/lucene/pull/12962#discussion_r1453309333


##
lucene/core/src/java/org/apache/lucene/search/TopKnnCollector.java:
##
@@ -27,25 +29,72 @@
  */
 public final class TopKnnCollector extends AbstractKnnCollector {
 
+  // greediness of globally non-competitive search: [0,1]
+  private static final float DEFAULT_GREEDINESS = 0.9f;
   private final NeighborQueue queue;
+  private final float greediness;
+  private final FloatHeap nonCompetitiveQueue;
+  private final FloatHeap updatesQueue;
+  private final int interval = 0x3ff; // 1023

Review Comment:
   This seems quite large to me, based on the total node visited counts in the 
Cohere results. For example, for fo = 90 we'd only refresh around twice per 
segment.



-- 
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] Speedup concurrent multi-segment HNWS graph search 2 [lucene]

2024-01-16 Thread via GitHub


tveasey commented on code in PR #12962:
URL: https://github.com/apache/lucene/pull/12962#discussion_r1453323532


##
lucene/core/src/java/org/apache/lucene/search/TopKnnCollector.java:
##
@@ -27,25 +29,72 @@
  */
 public final class TopKnnCollector extends AbstractKnnCollector {
 
+  // greediness of globally non-competitive search: [0,1]
+  private static final float DEFAULT_GREEDINESS = 0.9f;
   private final NeighborQueue queue;
+  private final float greediness;
+  private final FloatHeap nonCompetitiveQueue;
+  private final FloatHeap updatesQueue;
+  private final int interval = 0x3ff; // 1023
+  private final BlockingFloatHeap globalSimilarityQueue;
+  private boolean kResultsCollected = false;
+  private float cachedGlobalMinSim = Float.NEGATIVE_INFINITY;
 
   /**
* @param k the number of neighbors to collect
* @param visitLimit how many vector nodes the results are allowed to visit
*/
-  public TopKnnCollector(int k, int visitLimit) {
+  public TopKnnCollector(int k, int visitLimit, BlockingFloatHeap 
globalSimilarityQueue) {
 super(k, visitLimit);
+this.greediness = DEFAULT_GREEDINESS;
 this.queue = new NeighborQueue(k, false);
+this.globalSimilarityQueue = globalSimilarityQueue;
+
+if (globalSimilarityQueue == null) {
+  this.nonCompetitiveQueue = null;
+  this.updatesQueue = null;
+} else {
+  this.nonCompetitiveQueue = new FloatHeap(Math.max(1, Math.round((1 - 
greediness) * k)));
+  this.updatesQueue = new FloatHeap(k);
+}
   }
 
   @Override
   public boolean collect(int docId, float similarity) {
-return queue.insertWithOverflow(docId, similarity);
+boolean localSimUpdated = queue.insertWithOverflow(docId, similarity);
+boolean firstKResultsCollected = (kResultsCollected == false && 
queue.size() == k());
+if (firstKResultsCollected) {
+  kResultsCollected = true;
+}
+
+boolean globalSimUpdated = false;
+if (globalSimilarityQueue != null) {
+  updatesQueue.offer(similarity);
+  globalSimUpdated = nonCompetitiveQueue.offer(similarity);
+
+  if (kResultsCollected) {
+// as we've collected k results, we can start do periodic updates with 
the global queue
+if (firstKResultsCollected || (visitedCount & interval) == 0) {
+  cachedGlobalMinSim = 
globalSimilarityQueue.offer(updatesQueue.getHeap());

Review Comment:
   There is a subtlety: note that `firstKResultsCollected` is only true the 
first time we visit k nodes, so this is saying only refresh every 1024 
iterations thereafter. The refresh schedule is `k, 1024, 2048, ...`. (The || 
threw me initially.)
   
   As per my comment above, 1024 seems infrequent to me. (Of course you may 
have tested smaller values and determined this to be a good choice.) If we 
think it is risky sharing information too early, I would be inclined to share 
on the schedule `max(k, c1), max(k, c1) + c2, max(k, c1) + 2 * c2, ...` with 
`c1 > c2` and decouple the concerns.
   
   Also, there maybe issues with *using* information too early, in which case 
`minCompetitiveSimilarity` would need a check that `visitedCount > max(k, c1)` 
before it starts modifying `minSim`.
   
   In any case, I can see results being rather sensitive to these choices so if 
you haven't done a parameter exploration it might be worth trying a few choices.



-- 
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] Speedup concurrent multi-segment HNWS graph search 2 [lucene]

2024-01-16 Thread via GitHub


tveasey commented on code in PR #12962:
URL: https://github.com/apache/lucene/pull/12962#discussion_r1453341457


##
lucene/core/src/java/org/apache/lucene/util/hnsw/BlockingFloatHeap.java:
##
@@ -0,0 +1,190 @@
+/*
+ * 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.util.hnsw;
+
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A blocking bounded min heap that stores floats. The top element is the 
lowest value of the heap.
+ *
+ * A primitive priority queue that maintains a partial ordering of its 
elements such that the
+ * least element can always be found in constant time. Implementation is based 
on {@link
+ * org.apache.lucene.util.LongHeap}
+ *
+ * @lucene.internal
+ */
+public final class BlockingFloatHeap {
+  private final int maxSize;
+  private final float[] heap;
+  private final ReentrantLock lock;
+  private int size;
+
+  public BlockingFloatHeap(int maxSize) {
+this.maxSize = maxSize;
+this.heap = new float[maxSize + 1];
+this.lock = new ReentrantLock();
+this.size = 0;
+  }
+
+  /**
+   * Inserts a value into this heap.
+   *
+   * If the number of values would exceed the heap's maxSize, the least 
value is discarded
+   *
+   * @param value the value to add
+   * @return the new 'top' element in the queue.
+   */
+  public float offer(float value) {
+lock.lock();
+try {
+  if (size < maxSize) {
+push(value);
+return heap[1];
+  } else {
+if (value >= heap[1]) {
+  updateTop(value);
+}
+return heap[1];
+  }
+} finally {
+  lock.unlock();
+}
+  }
+
+  /**
+   * Inserts array of values into this heap.
+   *
+   * Values are expected to be sorted in ascending order.
+   *
+   * @param values a set of values to insert
+   * @return the new 'top' element in the queue.
+   */
+  public float offer(float[] values) {
+lock.lock();
+try {
+  for (int i = values.length - 1; i >= 0; i--) {
+if (size < maxSize) {
+  push(values[i]);
+} else {
+  if (values[i] >= heap[1]) {

Review Comment:
   This should break otherwise if the values are sorted.



-- 
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] join: repeat BytesRefHash.sort() in TermsQuery after TermsIncludingScoreQuery [lucene]

2024-01-16 Thread via GitHub


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

   ### Description
   
   * The `TermsIncludingScoreQuery` constructor as per 
https://github.com/apache/lucene/blob/releases/lucene/9.9.1/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java#L74
 does a `BytesRefHash.sort()` call on the `terms` object passed in to it.
   * Subsequently `TermsIncludingScoreQuery.createWeight` as per 
https://github.com/apache/lucene/blob/releases/lucene/9.9.1/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java#L134
 passes the `terms` object to the `TermsQuery` constructor.
   * The `TermsQuery` constructor as per 
https://github.com/apache/lucene/blob/releases/lucene/9.9.1/lucene/join/src/java/org/apache/lucene/search/join/TermsQuery.java#L68
 also does a `BytesRefHash.sort()` call on the `terms` object passed in to it.
   
   The current Lucene tests pass but on the dev mailing as per 
https://lists.apache.org/thread/ms65gkmm86cpp5ntdo555ob8mrgg64o5 test failures 
for `org.apache.solr.search.join.ScoreJoinQParserPlugin` were reported and it 
was mentioned that after the #12784 changes `BytesRefHash#sort` may no longer 
be called more than once.
   
   
   ### Version and environment details
   
   _No response_


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



[PR] join: avoid repeat BytesRefHash.sort() in TermsQuery after TermsIncludingScoreQuery [lucene]

2024-01-16 Thread via GitHub


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

   naive change for the #13018 issue.
   
   TODOs -- collaboration and pushes to the PR branch welcome
   * test coverage
   * confirmation that this would solve the 
`org.apache.solr.search.join.ScoreJoinQParserPlugin` issues
   * ideas on how callers might workaround the issue using 9.9.1 Lucene


-- 
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] join: repeat BytesRefHash.sort() in TermsQuery after TermsIncludingScoreQuery [lucene]

2024-01-16 Thread via GitHub


cpoerschke commented on issue #13018:
URL: https://github.com/apache/lucene/issues/13018#issuecomment-1894343041

   Just spotted that @gf2121's #13014 is also about this? Great, thanks!


-- 
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] Speedup concurrent multi-segment HNWS graph search 2 [lucene]

2024-01-16 Thread via GitHub


mayya-sharipova commented on PR #12962:
URL: https://github.com/apache/lucene/pull/12962#issuecomment-1894402548

   I have done more experiments with different `interval` values: 
   
   Cohere 786 dims:
   
   1M vectors, k=10, fanout=90
   |  Interval  | Avg visited nodes |   QPS|   Recall| 
   |  ---:  |---:   | ---: |---: |
   | 255|  10283|   959|0.946|
   | 511|  11455|   912|0.956|
   | 1023   |  12241|   888|0.963|
   
   
   1M vectors, k=100, fanout=900
   |  Interval  | Avg visited nodes |   QPS|   Recall| 
   |  ---:  |---:   | ---: |---: |  
   | 255|  28175|   286|0.978|
   | 511|  32410|   231|0.982|
   | 1023   |  38929|   193|0.985|
   
   
   10M vectors, k=10, fanout=90
   |  Interval  | Avg visited nodes |   QPS|   Recall| 
   |  ---:  |---:   | ---: |---: |   
   | 255|  18325|   567|0.906|
   | 511|  20108|   518|0.919|
   | 1023   |  22041|   472|0.932|
   
   10M vectors, k=100, fanout=900
   |  Interval  | Avg visited nodes |   QPS|   Recall| 
   |  ---:  |---:   | ---: |---: |
   | 255|  58543|   157|0.958|
   | 511|  66518|   134|0.966|
   | 1023   |  78740|   115|0.972|
   
   


-- 
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] `SynonymGraphFilter` should read FSTs off-heap? [lucene]

2024-01-16 Thread via GitHub


msfroh commented on issue #13005:
URL: https://github.com/apache/lucene/issues/13005#issuecomment-1894635526

   I was looking into how to implement this and I think I've mostly got it -- 
essentially, I would write the `SynonymMap` to a file (which could be an 
offline operation, basically "precompile your SynonymMap" so you don't need to 
parse + compile it on startup).
   
   What's got me stuck is that `OffHeapFSTStore` takes an `IndexInput`, which 
AFAIK should only be returned from a `Directory`. We don't want to write the 
`SynonymMap` to the index where it's used, right?
   
   Huh... I guess we could use a separate, sidecar directory for the 
precompiled SynonymMap. That directory could optionally be passed to 
`SynonymGraphFilterFactory` to let it load a precompiled (off-heap) 
`SynonymMap`.


-- 
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] LUCENE-10641: IndexSearcher#setTimeout should also abort query rewrites, point ranges and vector searches [lucene]

2024-01-16 Thread via GitHub


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

   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] Split taxonomy arrays across chunks [lucene]

2024-01-16 Thread via GitHub


msfroh commented on code in PR #12995:
URL: https://github.com/apache/lucene/pull/12995#discussion_r1454250264


##
lucene/facet/src/test/org/apache/lucene/facet/taxonomy/TestTaxonomyCombined.java:
##
@@ -669,20 +668,26 @@ public void testChildrenArraysInvariants() throws 
Exception {
   // Find the youngest older sibling:
   int j;
   for (j = i - 1; j >= 0; j--) {
-if (parents[j] == parents[i]) {
+if (parents.get(j) == parents.get(i)) {
   break; // found youngest older sibling
 }
   }
   if (j < 0) { // no sibling found
 j = TaxonomyReader.INVALID_ORDINAL;
   }
-  assertEquals(j, olderSiblingArray[i]);
+  assertEquals(j, olderSiblingArray.get(i));
 }
 
 tr.close();
 indexDir.close();
   }
 
+  private static void assertArrayEquals(int[] expected, 
ParallelTaxonomyArrays.IntArray actual) {

Review Comment:
   This was the only place where we assert anything about the full contents of 
the `ChunkedArray`. I would lean toward not adding `equals` to `ChunkedArray`, 
since it's not something we would normally want to call, IMO. (In general, it's 
going to be pretty expensive.)



-- 
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] Split taxonomy arrays across chunks [lucene]

2024-01-16 Thread via GitHub


msfroh commented on code in PR #12995:
URL: https://github.com/apache/lucene/pull/12995#discussion_r1454324230


##
lucene/facet/src/java/org/apache/lucene/facet/taxonomy/directory/TaxonomyIndexArrays.java:
##
@@ -68,25 +90,49 @@ public TaxonomyIndexArrays(IndexReader reader, 
TaxonomyIndexArrays copyFrom) thr
 // it may be caused if e.g. the taxonomy segments were merged, and so an 
updated
 // NRT reader was obtained, even though nothing was changed. this is not 
very likely
 // to happen.
-int[] copyParents = copyFrom.parents();
-this.parents = new int[reader.maxDoc()];
-System.arraycopy(copyParents, 0, parents, 0, copyParents.length);
-initParents(reader, copyParents.length);
-
+int[][] parentArray = allocateChunkedArray(reader.maxDoc());
+copyChunkedArray(copyFrom.parents.values, parentArray);
+initParents(parentArray, reader, copyFrom.parents.length());
+parents = new ChunkedArray(parentArray);
 if (copyFrom.initializedChildren) {
   initChildrenSiblings(copyFrom);
 }
   }
 
+  private static int[][] allocateChunkedArray(int size) {
+int chunkCount = size / CHUNK_SIZE + 1;

Review Comment:
   Got it! 
   
   I handled the multiple of `CHUNK_SIZE` case without allocating an empty 
array. I also added a unit test that specifically flexes ChunkedIntArray.
   
   (And I cleaned up a few more cases where I was still dividing/moduloing 
instead of bit-shifting and masking.)



-- 
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] Speedup concurrent multi-segment HNWS graph search 2 [lucene]

2024-01-16 Thread via GitHub


mayya-sharipova commented on code in PR #12962:
URL: https://github.com/apache/lucene/pull/12962#discussion_r1454345777


##
lucene/core/src/java/org/apache/lucene/util/hnsw/BlockingFloatHeap.java:
##
@@ -0,0 +1,190 @@
+/*
+ * 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.util.hnsw;
+
+import java.util.concurrent.locks.ReentrantLock;
+
+/**
+ * A blocking bounded min heap that stores floats. The top element is the 
lowest value of the heap.
+ *
+ * A primitive priority queue that maintains a partial ordering of its 
elements such that the
+ * least element can always be found in constant time. Implementation is based 
on {@link
+ * org.apache.lucene.util.LongHeap}
+ *
+ * @lucene.internal
+ */
+public final class BlockingFloatHeap {
+  private final int maxSize;
+  private final float[] heap;
+  private final ReentrantLock lock;
+  private int size;
+
+  public BlockingFloatHeap(int maxSize) {
+this.maxSize = maxSize;
+this.heap = new float[maxSize + 1];
+this.lock = new ReentrantLock();
+this.size = 0;
+  }
+
+  /**
+   * Inserts a value into this heap.
+   *
+   * If the number of values would exceed the heap's maxSize, the least 
value is discarded
+   *
+   * @param value the value to add
+   * @return the new 'top' element in the queue.
+   */
+  public float offer(float value) {
+lock.lock();
+try {
+  if (size < maxSize) {
+push(value);
+return heap[1];
+  } else {
+if (value >= heap[1]) {
+  updateTop(value);
+}
+return heap[1];
+  }
+} finally {
+  lock.unlock();
+}
+  }
+
+  /**
+   * Inserts array of values into this heap.
+   *
+   * Values are expected to be sorted in ascending order.
+   *
+   * @param values a set of values to insert
+   * @return the new 'top' element in the queue.
+   */
+  public float offer(float[] values) {
+lock.lock();
+try {
+  for (int i = values.length - 1; i >= 0; i--) {
+if (size < maxSize) {
+  push(values[i]);
+} else {
+  if (values[i] >= heap[1]) {

Review Comment:
   @tveasey Thanks for your feedback. I am not sure what do you meant by 
"break".  In this method, it is good if the passed `values` are sorted in asc 
order, but it is not necessary.
   
   This method is for bulk update of the queue. For a faster performance it is 
better to start with larger values, so that if `values[i] < heap[1]`, we don't 
need to do anything.



-- 
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] SegmentDocValuesProducer checkIntegrity might open a dropped segment [lucene]

2024-01-16 Thread via GitHub


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

   ### Description
   
   We are using HDFS for file storage and the softUpdateDocuments interface for 
writing data. 
   We have found that during concurrent writes, the dvd files selected for 
merging can be deleted by other write/flush threads
   If we change to updateDocuments interface for writing data, 
FileNotFoundException will not occur.
   we test lucene-9.5.0 to lucene-9.8.0, all theses version will occur this 
exception.
   
   The exception as follows:
   java.io.FileNotFoundException: File does not exist: 
/search/test/1/index/_l5_1_Lucene90_0.dvd
   at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:2308)
   at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:800)
   at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB.java:479)
   at 
org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$ClientNamenodeProtocol$2.callBlockingMethod(ClientNamenodeProtocolProtos.java)
   
   at 
org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(DFSClient.java:1403) 
   at 
org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1390) 
   at 
org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(DFSClient.java:1379) 
   at 
org.apache.hadoop.hdfs.DFSInputStream.fetchLocatedBlocksAndGetLastBlockLength(DFSInputStream.java:366)
 
   at 
org.apache.hadoop.hdfs.DFSInputStream.openInfo(DFSInputStream.java:284) 
   at 
org.apache.hadoop.hdfs.DFSInputStream.refetchLocations(DFSInputStream.java:1299)
 
   at 
org.apache.hadoop.hdfs.DFSInputStream.chooseDataNode(DFSInputStream.java:1245) 
   at 
org.apache.hadoop.hdfs.DFSInputStream.chooseDataNode(DFSInputStream.java:1224) 
   at 
org.apache.hadoop.hdfs.DFSInputStream.fetchBlockByteRange(DFSInputStream.java:1405)
 
   at 
org.apache.hadoop.hdfs.DFSInputStream.doPread(DFSInputStream.java:1831) 
   at 
org.apache.hadoop.hdfs.DFSInputStream.pread(DFSInputStream.java:1785) 
   at 
org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:1773) 
   at 
org.apache.hadoop.fs.FSInputStream.readFully(FSInputStream.java:124) 
   at 
org.apache.hadoop.fs.FSDataInputStream.readFully(FSDataInputStream.java:117) 
   
   at org.apache.lucene.store.DataInput.readBytes(DataInput.java:72)
   at 
org.apache.lucene.store.ChecksumIndexInput.skipByReading(ChecksumIndexInput.java:79)
   at 
org.apache.lucene.store.ChecksumIndexInput.seek(ChecksumIndexInput.java:64)
   at 
org.apache.lucene.codecs.CodecUtil.checksumEntireFile(CodecUtil.java:618)
   at 
org.apache.lucene.codecs.lucene90.Lucene90DocValuesProducer.checkIntegrity(Lucene90DocValuesProducer.java:1640)
 
   at 
org.apache.lucene.codecs.perfield.PerFieldDocValuesFormat$FieldsReader.checkIntegrity(PerFieldDocValuesFormat.java:380)
   at 
org.apache.lucene.index.SegmentDocValuesProducer.checkIntegrity(SegmentDocValuesProducer.java:131)
   
   Caused by: org.apache.hadoop.ipc.RemoteException: File does not exist: 
/search/test/1/index/_l5_1_Lucene90_0.dvd
   at 
org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:125)
   at 
org.apache.hadoop.hdfs.server.namenode.INodeFile.valueOf(INodeFile.java:115)
   at 
org.apache.hadoop.hdfs.server.namenode.FSDirStatAndListingOp.getBlockLocations(FSDirStatAndListingOp.java:205)
   at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem$1.doCall(FSNamesystem.java:2304)
   at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem$1.doCall(FSNamesystem.java:2301)
   at 
org.apache.hadoop.hdfs.server.namenode.LinkResolver.resolve(LinkResolver.java:43)
   at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.getBlockLocations(FSNamesystem.java:2308)
   at 
org.apache.hadoop.hdfs.server.namenode.NameNodeRpcServer.getBlockLocations(NameNodeRpcServer.java:800)
   at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolServerSideTranslatorPB.getBlockLocations(ClientNamenodeProtocolServerSideTranslatorPB.java:479)
   
   ### Version and environment details
   
   _No response_


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