[GitHub] [lucene] stefanvodita commented on a diff in pull request #12462: Fix stack overflow in RegExp for long string

2023-08-10 Thread via GitHub


stefanvodita commented on code in PR #12462:
URL: https://github.com/apache/lucene/pull/12462#discussion_r1290149550


##
lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java:
##
@@ -1067,22 +1071,44 @@ private boolean check(int flag) {
   }
 
   final RegExp parseUnionExp() throws IllegalArgumentException {
-RegExp e = parseInterExp();
-if (match('|')) e = makeUnion(flags, e, parseUnionExp());
-return e;
+return iterativeParseExp(this::parseInterExp, () -> match('|'), 
RegExp::makeUnion);
   }
 
   final RegExp parseInterExp() throws IllegalArgumentException {
-RegExp e = parseConcatExp();
-if (check(INTERSECTION) && match('&')) e = makeIntersection(flags, e, 
parseInterExp());
-return e;
+return iterativeParseExp(
+this::parseConcatExp, () -> check(INTERSECTION) && match('&'), 
RegExp::makeIntersection);
   }
 
   final RegExp parseConcatExp() throws IllegalArgumentException {
-RegExp e = parseRepeatExp();
-if (more() && !peek(")|") && (!check(INTERSECTION) || !peek("&")))
-  e = makeConcatenation(flags, e, parseConcatExp());
-return e;
+return iterativeParseExp(
+this::parseRepeatExp,
+() -> (more() && !peek(")|") && (!check(INTERSECTION) || !peek("&"))),
+RegExp::makeConcatenation);
+  }
+
+  /**
+   * Custom Functional Interface for a Supplying methods with signature of 
RegExp(int int1, RegExp
+   * exp1, RegExp exp2)
+   */
+  @FunctionalInterface
+  private interface MakeRegexGroup {
+RegExp get(int int1, RegExp exp1, RegExp exp2);
+  }
+
+  final RegExp iterativeParseExp(
+  Supplier gather, BooleanSupplier stop, MakeRegexGroup reduce)
+  throws IllegalArgumentException {
+Deque regExpStack = new ArrayDeque<>();

Review Comment:
   Good point! In these cases, `reduce` is an associative function. If we do 
this, maybe we can rename it to `associativeReduce` or something similar to 
make this extra assumption obvious.



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



[GitHub] [lucene] slow-J commented on a diff in pull request #12462: Fix stack overflow in RegExp for long string

2023-08-10 Thread via GitHub


slow-J commented on code in PR #12462:
URL: https://github.com/apache/lucene/pull/12462#discussion_r1290188698


##
lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java:
##
@@ -1067,22 +1071,44 @@ private boolean check(int flag) {
   }
 
   final RegExp parseUnionExp() throws IllegalArgumentException {
-RegExp e = parseInterExp();
-if (match('|')) e = makeUnion(flags, e, parseUnionExp());
-return e;
+return iterativeParseExp(this::parseInterExp, () -> match('|'), 
RegExp::makeUnion);
   }
 
   final RegExp parseInterExp() throws IllegalArgumentException {
-RegExp e = parseConcatExp();
-if (check(INTERSECTION) && match('&')) e = makeIntersection(flags, e, 
parseInterExp());
-return e;
+return iterativeParseExp(
+this::parseConcatExp, () -> check(INTERSECTION) && match('&'), 
RegExp::makeIntersection);
   }
 
   final RegExp parseConcatExp() throws IllegalArgumentException {
-RegExp e = parseRepeatExp();
-if (more() && !peek(")|") && (!check(INTERSECTION) || !peek("&")))
-  e = makeConcatenation(flags, e, parseConcatExp());
-return e;
+return iterativeParseExp(
+this::parseRepeatExp,
+() -> (more() && !peek(")|") && (!check(INTERSECTION) || !peek("&"))),
+RegExp::makeConcatenation);
+  }
+
+  /**
+   * Custom Functional Interface for a Supplying methods with signature of 
RegExp(int int1, RegExp
+   * exp1, RegExp exp2)
+   */
+  @FunctionalInterface
+  private interface MakeRegexGroup {
+RegExp get(int int1, RegExp exp1, RegExp exp2);
+  }
+
+  final RegExp iterativeParseExp(
+  Supplier gather, BooleanSupplier stop, MakeRegexGroup reduce)
+  throws IllegalArgumentException {
+Deque regExpStack = new ArrayDeque<>();

Review Comment:
   Thanks for the suggestion Patrick and Stefan.
   
   As long as we do the suggested `reduce.get(flags, res, e)` instead of 
`reduce.get(flags, e, res)` the result should be correct.
   
   Will rename `reduce` to highlight it's associativeness.



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



[GitHub] [lucene] slow-J commented on a diff in pull request #12462: Fix stack overflow in RegExp for long string

2023-08-10 Thread via GitHub


slow-J commented on code in PR #12462:
URL: https://github.com/apache/lucene/pull/12462#discussion_r1290219209


##
lucene/core/src/java/org/apache/lucene/util/automaton/RegExp.java:
##
@@ -1067,22 +1071,44 @@ private boolean check(int flag) {
   }
 
   final RegExp parseUnionExp() throws IllegalArgumentException {
-RegExp e = parseInterExp();
-if (match('|')) e = makeUnion(flags, e, parseUnionExp());
-return e;
+return iterativeParseExp(this::parseInterExp, () -> match('|'), 
RegExp::makeUnion);
   }
 
   final RegExp parseInterExp() throws IllegalArgumentException {
-RegExp e = parseConcatExp();
-if (check(INTERSECTION) && match('&')) e = makeIntersection(flags, e, 
parseInterExp());
-return e;
+return iterativeParseExp(
+this::parseConcatExp, () -> check(INTERSECTION) && match('&'), 
RegExp::makeIntersection);
   }
 
   final RegExp parseConcatExp() throws IllegalArgumentException {
-RegExp e = parseRepeatExp();
-if (more() && !peek(")|") && (!check(INTERSECTION) || !peek("&")))
-  e = makeConcatenation(flags, e, parseConcatExp());
-return e;
+return iterativeParseExp(
+this::parseRepeatExp,
+() -> (more() && !peek(")|") && (!check(INTERSECTION) || !peek("&"))),
+RegExp::makeConcatenation);
+  }
+
+  /**
+   * Custom Functional Interface for a Supplying methods with signature of 
RegExp(int int1, RegExp
+   * exp1, RegExp exp2)
+   */
+  @FunctionalInterface
+  private interface MakeRegexGroup {
+RegExp get(int int1, RegExp exp1, RegExp exp2);
+  }
+
+  final RegExp iterativeParseExp(
+  Supplier gather, BooleanSupplier stop, MakeRegexGroup reduce)
+  throws IllegalArgumentException {
+Deque regExpStack = new ArrayDeque<>();

Review Comment:
   Added a new commit with the 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



[GitHub] [lucene] javanna opened a new issue, #12498: Simplify task executor for concurrent operations

2023-08-10 Thread via GitHub


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

   `IndexSearcher` supports parallelizing search across slices when an executor 
is available. Knn queries can also parallelize their rewrite across segments 
using the same executor. Potentially, other operations will be parallelized in 
the future using the same pattern.
   
   Lucene currently has the notion of a `TaskExecutor` (previously named 
`SliceExecutor`, but because knn rewrite is parallelized across segments rather 
than slices, it was recently renamed to `TaskExecutor`) which is responsible 
for offloading tasks to the executor, wait for them all to complete and return 
the corresponding results.
   
   IndexSearcher currently has an [instanceof 
check](https://github.com/apache/lucene/blob/main/lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java#L1011)
 of the provided executor, and if it's a `ThreadPoolExecutor`, which is likely, 
a `QueueSizeBasedExecutor` is used which tries to be smart about when to 
offload tasks to the executor vs when to execute them on the caller thread. 
This behaviour is not configurable in any way.
   
   As part of exposing concurrent search in Elasticsearch, we have found that 
the current task executor is too opinionated, and either it needs to be 
customizable, or simplified to be less opinionated.
   
   In a server-side search engine, the search itself is likely executed by a 
thread pool that has its own sizing, queueing mechanisms as well as rejection 
policy. When offloading requests to an executor for additional parallelism, it 
is important to be able to determine where the load is going to be and what 
type of workload it maps to. Ideally, the caller thread is merely coordinating 
and not doing any I/O nor CPU intensive work, that is instead all delegated to 
the separate worker threads. Having built-in rules for when to execute certain 
operations on the caller thread may cause more problems than it solves, as it 
is unpredictable and makes sizing of thread pools more difficult, because all 
of a sudden you end up with two thread pools that may execute I/O as well as 
CPU intensive operations.
   
   My conclusion is that if flexibility is needed in terms of possibly 
executing on the caller thread, such behaviour can be included in the executor 
that is provided to the searcher (for example with an adaptive mechanism that 
conditionally executes directly instead of offloading based on queue size like 
`QueueSizeBasedExecutor` does), together with its saturation policy (as opposed 
to catching `RejectedExecutionException` and executing on the caller thread, 
which is potentially dangerous). Also, executing the last slice / task on the 
caller thread, as it's the one waiting for all the tasks to complete, is not 
necessarily addressing a real problem around under-utilization of a thread that 
is doing nothing. That wait is cheap, and it's possibly more important to 
divide the workload among the thread pools. That said, such behavior can also 
be included in the `Executor` itself and does not require additional extension 
points.
   
   My proposal is that we remove the `QueueSizeBasedExecutor` entirely and we 
simply offload every task to the executor, unconditionally. It's up to the 
provided executor to determine what to do when execute is called. That is the 
pluggable behaviour. Lucene should not have strong opinions nor provide 
building blocks for how to execute concurrent tasks.
   
   Additionally, I think that we should unconditionally offload execution to 
the executor when available, even when we have a single slice. It may seem 
counter intuitive but it's again to be able to determine what type of workload 
each thread pool performs.


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



[GitHub] [lucene] javanna commented on issue #12498: Simplify task executor for concurrent operations

2023-08-10 Thread via GitHub


javanna commented on issue #12498:
URL: https://github.com/apache/lucene/issues/12498#issuecomment-1673363696

   @jpountz @mikemccand pinging you two because you are likely to have thoughts 
on this.


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



[GitHub] [lucene] javanna opened a new pull request, #12499: Simplify task executor for concurrent operations

2023-08-10 Thread via GitHub


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

   Based on the proposal in #12498, this PR removes the QueueSizeBasedExecutor 
(package private) in favour of simply offloading concurrent execution to the 
provided executor. In need of specific behaviour, it can all be included in the 
executor itself.
   
   This removes an instanceof check that determines which type of executor 
wrapper is used, which means that some tasks may be executed on the caller 
thread depending on queue size, whenever a rejection happens, or always for the 
last slice. This behaviour is not configurable in any way, and is too rigid. 
Rather than making this pluggable, I propose to make Lucene less opinionated 
about concurrent tasks execution and require that users include their own 
execution strategy directly in the executor that they provide to the index 
searcher.
   
   Relates to #12498


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



[GitHub] [lucene] shubhamvishu commented on pull request #12428: Replace consecutive close() calls and close() calls with null checks with IOUtils.close()

2023-08-10 Thread via GitHub


shubhamvishu commented on PR #12428:
URL: https://github.com/apache/lucene/pull/12428#issuecomment-1673477487

   @gsmiller I have added the CHANGES entry under 9.8.0 and replaced a couple 
of more such calls I could find. Let me know if this looks good. 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



[GitHub] [lucene] shubhamvishu commented on pull request #12427: StringsToAutomaton#build to take List as parameter instead of Collection

2023-08-10 Thread via GitHub


shubhamvishu commented on PR #12427:
URL: https://github.com/apache/lucene/pull/12427#issuecomment-1673498218

   > I still think the explicit validation is better in this situation, but the 
counter-argument would be a user that knows for sure they're always properly 
providing sorted input and don't want the overhead of the validation.
   
   Agreed! As we discussed above one way could be to sort based on user input 
that if the collection is sorted or not but since that will be for only one 
flavour it might be confusing. I'll try to benchmark this change to see if its 
causing any regression or not.
   
   > I think it's so that next can throw IOException? You can't do that with 
Iterator.
   
   Ahh I see. That makes sense. Thanks @gsmiller !


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



[GitHub] [lucene] tang-hi commented on pull request #12472: Fix UTF32toUTF8 will produce invalid transition

2023-08-10 Thread via GitHub


tang-hi commented on PR #12472:
URL: https://github.com/apache/lucene/pull/12472#issuecomment-1673497452

   @mikemccand Just a gentle reminder, perhaps you can review this code and 
then move this PR forward.


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



[GitHub] [lucene] jbellis commented on a diff in pull request #12421: Concurrent hnsw graph and builder, take two

2023-08-10 Thread via GitHub


jbellis commented on code in PR #12421:
URL: https://github.com/apache/lucene/pull/12421#discussion_r1290379055


##
lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java:
##
@@ -104,7 +103,9 @@ public void addNode(int level, int node) {
   // and make this node the graph's new entry point
   if (level >= numLevels) {
 for (int i = numLevels; i <= level; i++) {
-  graphUpperLevels.add(new HashMap<>());
+  graphUpperLevels.add(
+  new HashMap<>(
+  16, levelLoadFactor)); // these are the default parameters, 
made explicit
 }

Review Comment:
   We specify the loadFactor because it's necessary to compute ram usage, and 
you can't retrieve the loadFactor once the Map is constructed.  
   
   The 16 is there because before it was just saying new HashMap() and I can't 
specify the loadFactor w/o also specifying the initialCapacity.  As the comment 
indicates, 16 is the default initialCapacity.



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



[GitHub] [lucene] jbellis commented on a diff in pull request #12421: Concurrent hnsw graph and builder, take two

2023-08-10 Thread via GitHub


jbellis commented on code in PR #12421:
URL: https://github.com/apache/lucene/pull/12421#discussion_r1290387939


##
lucene/core/src/java/org/apache/lucene/util/hnsw/OnHeapHnswGraph.java:
##
@@ -169,35 +170,43 @@ public NodesIterator getNodesOnLevel(int level) {
 
   @Override
   public long ramBytesUsed() {
+// local vars here just to make it easier to keep lines short enough to 
read
+long AH_BYTES = RamUsageEstimator.NUM_BYTES_ARRAY_HEADER;
+long REF_BYTES = RamUsageEstimator.NUM_BYTES_OBJECT_REF;
+
 long neighborArrayBytes0 =
-nsize0 * (Integer.BYTES + Float.BYTES)
-+ RamUsageEstimator.NUM_BYTES_ARRAY_HEADER
-+ RamUsageEstimator.NUM_BYTES_OBJECT_REF * 2
-+ Integer.BYTES * 3;
+(long) nsize0 * (Integer.BYTES + Float.BYTES)
++ AH_BYTES * 2
++ REF_BYTES
++ Integer.BYTES * 2;

Review Comment:
   i don't think so, but it was missing 1 byte for the boolean.  I've added 
comments to clarify



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



[GitHub] [lucene] jbellis commented on a diff in pull request #12421: Concurrent hnsw graph and builder, take two

2023-08-10 Thread via GitHub


jbellis commented on code in PR #12421:
URL: https://github.com/apache/lucene/pull/12421#discussion_r1290397717


##
lucene/core/src/java/org/apache/lucene/util/hnsw/NeighborArray.java:
##
@@ -126,12 +124,16 @@ private int insertSortedInternal() {
 return insertionPoint;
   }
 
-  /** This method is for test only. */
-  void insertSorted(int newNode, float newScore) {
+  protected void insertSorted(int newNode, float newScore) {

Review Comment:
   see the comments to ConcurrentNeighborArray.insertSorted



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



[GitHub] [lucene] jbellis commented on a diff in pull request #12421: Concurrent hnsw graph and builder, take two

2023-08-10 Thread via GitHub


jbellis commented on code in PR #12421:
URL: https://github.com/apache/lucene/pull/12421#discussion_r1290402830


##
lucene/core/src/java/org/apache/lucene/util/hnsw/ConcurrentNeighborSet.java:
##
@@ -0,0 +1,292 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.PrimitiveIterator;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.FixedBitSet;
+
+/** A concurrent set of neighbors. */
+public class ConcurrentNeighborSet {
+  /** the node id whose neighbors we are storing */
+  private final int nodeId;
+
+  /**
+   * We use a copy-on-write NeighborArray to store the neighbors. Even though 
updating this is
+   * expensive, it is still faster than using a concurrent Collection because 
"iterate through a
+   * node's neighbors" is a hot loop in adding to the graph, and NeighborArray 
can do that much
+   * faster: no boxing/unboxing, all the data is stored sequentially instead 
of having to follow
+   * references, and no fancy encoding necessary for node/score.
+   */
+  private final AtomicReference neighborsRef;
+
+  private final NeighborSimilarity similarity;
+
+  /** the maximum number of neighbors we can store */
+  private final int maxConnections;
+
+  public ConcurrentNeighborSet(int nodeId, int maxConnections, 
NeighborSimilarity similarity) {
+this.nodeId = nodeId;
+this.maxConnections = maxConnections;
+this.similarity = similarity;
+neighborsRef = new AtomicReference<>(new 
ConcurrentNeighborArray(maxConnections, true));
+  }
+
+  public PrimitiveIterator.OfInt nodeIterator() {
+// don't use a stream here. stream's implementation of iterator buffers
+// very aggressively, which is a big waste for a lot of searches.
+return new NeighborIterator(neighborsRef.get());
+  }
+
+  public void backlink(Function 
neighborhoodOf) throws IOException {
+NeighborArray neighbors = neighborsRef.get();
+for (int i = 0; i < neighbors.size(); i++) {
+  int nbr = neighbors.node[i];
+  float nbrScore = neighbors.score[i];
+  ConcurrentNeighborSet nbrNbr = neighborhoodOf.apply(nbr);
+  nbrNbr.insert(nodeId, nbrScore);
+}
+  }
+
+  private static class NeighborIterator implements PrimitiveIterator.OfInt {
+private final NeighborArray neighbors;
+private int i;
+
+private NeighborIterator(NeighborArray neighbors) {
+  this.neighbors = neighbors;
+  i = 0;
+}
+
+@Override
+public boolean hasNext() {
+  return i < neighbors.size();
+}
+
+@Override
+public int nextInt() {
+  return neighbors.node[i++];
+}
+  }
+
+  public int size() {
+return neighborsRef.get().size();
+  }
+
+  public int arrayLength() {
+return neighborsRef.get().node.length;
+  }
+
+  /**
+   * For each candidate (going from best to worst), select it only if it is 
closer to target than it
+   * is to any of the already-selected candidates. This is maintained whether 
those other neighbors
+   * were selected by this method, or were added as a "backlink" to a node 
inserted concurrently
+   * that chose this one as a neighbor.
+   */
+  public void insertDiverse(NeighborArray candidates) {

Review Comment:
   insert and insertDiverse are the public api of the class, i don't think 
there's a reason to lock it down to package only



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



[GitHub] [lucene] jbellis commented on a diff in pull request #12421: Concurrent hnsw graph and builder, take two

2023-08-10 Thread via GitHub


jbellis commented on code in PR #12421:
URL: https://github.com/apache/lucene/pull/12421#discussion_r1290407428


##
lucene/core/src/java/org/apache/lucene/util/hnsw/ConcurrentNeighborSet.java:
##
@@ -0,0 +1,292 @@
+/*
+ * 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.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.PrimitiveIterator;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.FixedBitSet;
+
+/** A concurrent set of neighbors. */
+public class ConcurrentNeighborSet {
+  /** the node id whose neighbors we are storing */
+  private final int nodeId;
+
+  /**
+   * We use a copy-on-write NeighborArray to store the neighbors. Even though 
updating this is
+   * expensive, it is still faster than using a concurrent Collection because 
"iterate through a
+   * node's neighbors" is a hot loop in adding to the graph, and NeighborArray 
can do that much
+   * faster: no boxing/unboxing, all the data is stored sequentially instead 
of having to follow
+   * references, and no fancy encoding necessary for node/score.
+   */
+  private final AtomicReference neighborsRef;
+
+  private final NeighborSimilarity similarity;
+
+  /** the maximum number of neighbors we can store */
+  private final int maxConnections;
+
+  public ConcurrentNeighborSet(int nodeId, int maxConnections, 
NeighborSimilarity similarity) {
+this.nodeId = nodeId;
+this.maxConnections = maxConnections;
+this.similarity = similarity;
+neighborsRef = new AtomicReference<>(new 
ConcurrentNeighborArray(maxConnections, true));
+  }
+
+  public PrimitiveIterator.OfInt nodeIterator() {
+// don't use a stream here. stream's implementation of iterator buffers
+// very aggressively, which is a big waste for a lot of searches.
+return new NeighborIterator(neighborsRef.get());
+  }
+
+  public void backlink(Function 
neighborhoodOf) throws IOException {
+NeighborArray neighbors = neighborsRef.get();
+for (int i = 0; i < neighbors.size(); i++) {
+  int nbr = neighbors.node[i];
+  float nbrScore = neighbors.score[i];
+  ConcurrentNeighborSet nbrNbr = neighborhoodOf.apply(nbr);
+  nbrNbr.insert(nodeId, nbrScore);
+}
+  }
+
+  private static class NeighborIterator implements PrimitiveIterator.OfInt {
+private final NeighborArray neighbors;
+private int i;
+
+private NeighborIterator(NeighborArray neighbors) {
+  this.neighbors = neighbors;
+  i = 0;
+}
+
+@Override
+public boolean hasNext() {
+  return i < neighbors.size();
+}
+
+@Override
+public int nextInt() {
+  return neighbors.node[i++];
+}
+  }
+
+  public int size() {
+return neighborsRef.get().size();
+  }
+
+  public int arrayLength() {
+return neighborsRef.get().node.length;
+  }
+
+  /**
+   * For each candidate (going from best to worst), select it only if it is 
closer to target than it
+   * is to any of the already-selected candidates. This is maintained whether 
those other neighbors
+   * were selected by this method, or were added as a "backlink" to a node 
inserted concurrently
+   * that chose this one as a neighbor.
+   */
+  public void insertDiverse(NeighborArray candidates) {
+BitSet selected = new FixedBitSet(candidates.size());
+for (int i = candidates.size() - 1; i >= 0; i--) {
+  int cNode = candidates.node[i];
+  float cScore = candidates.score[i];
+  if (isDiverse(cNode, cScore, candidates, selected)) {
+selected.set(i);
+  }
+}
+insertMultiple(candidates, selected);
+// This leaves the paper's keepPrunedConnection option out; we might want 
to add that
+// as an option in the future.
+  }
+
+  private void insertMultiple(NeighborArray others, BitSet selected) {
+neighborsRef.getAndUpdate(
+current -> {
+  ConcurrentNeighborArray next = current.copy();

Review Comment:
   that would be a reasonable optimization, but this hasn't shown up as a hot 
spot in my profiling



-- 
This is an automated message from the Apache Git Service.
To respond 

[GitHub] [lucene] jbellis commented on a diff in pull request #12421: Concurrent hnsw graph and builder, take two

2023-08-10 Thread via GitHub


jbellis commented on code in PR #12421:
URL: https://github.com/apache/lucene/pull/12421#discussion_r1290415826


##
lucene/core/src/java/org/apache/lucene/util/hnsw/ConcurrentHnswGraphBuilder.java:
##
@@ -0,0 +1,468 @@
+/*
+ * 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 static java.lang.Math.log;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.lucene.index.VectorEncoding;
+import org.apache.lucene.index.VectorSimilarityFunction;
+import org.apache.lucene.util.GrowableBitSet;
+import org.apache.lucene.util.InfoStream;
+import org.apache.lucene.util.ThreadInterruptedException;
+import org.apache.lucene.util.hnsw.ConcurrentNeighborSet.NeighborSimilarity;
+import org.apache.lucene.util.hnsw.ConcurrentOnHeapHnswGraph.NodeAtLevel;
+
+/**
+ * Builder for Concurrent HNSW graph. See {@link HnswGraph} for a high level 
overview, and the
+ * comments to `addGraphNode` for details on the concurrent building approach.
+ *
+ * @param  the type of vector
+ */
+public class ConcurrentHnswGraphBuilder {
+
+  /** Default number of maximum connections per node */
+  public static final int DEFAULT_MAX_CONN = 16;
+
+  /**
+   * Default number of the size of the queue maintained while searching during 
a graph construction.
+   */
+  public static final int DEFAULT_BEAM_WIDTH = 100;
+
+  /** A name for the HNSW component for the info-stream */
+  public static final String HNSW_COMPONENT = "HNSW";
+
+  private final int beamWidth;
+  private final double ml;
+  private final ExplicitThreadLocal scratchNeighbors;
+
+  private final VectorSimilarityFunction similarityFunction;
+  private final VectorEncoding vectorEncoding;
+  private final ExplicitThreadLocal> vectors;
+  private final ExplicitThreadLocal> graphSearcher;
+  private final ExplicitThreadLocal beamCandidates;
+
+  final ConcurrentOnHeapHnswGraph hnsw;
+  private final ConcurrentSkipListSet insertionsInProgress =
+  new ConcurrentSkipListSet<>();
+
+  private InfoStream infoStream = InfoStream.getDefault();
+
+  // we need two sources of vectors in order to perform diversity check 
comparisons without
+  // colliding
+  private final ExplicitThreadLocal> vectorsCopy;
+
+  /** This is the "native" factory for ConcurrentHnswGraphBuilder. */
+  public static  ConcurrentHnswGraphBuilder create(
+  RandomAccessVectorValues vectors,
+  VectorEncoding vectorEncoding,
+  VectorSimilarityFunction similarityFunction,
+  int M,
+  int beamWidth)
+  throws IOException {
+return new ConcurrentHnswGraphBuilder<>(
+vectors, vectorEncoding, similarityFunction, M, beamWidth);
+  }
+
+  /**
+   * Reads all the vectors from vector values, builds a graph connecting them 
by their dense
+   * ordinals, using the given hyperparameter settings, and returns the 
resulting graph.
+   *
+   * @param vectorValues the vectors whose relations are represented by the 
graph - must provide a
+   * different view over those vectors than the one used to add via 
addGraphNode.
+   * @param M – graph fanout parameter used to calculate the maximum number of 
connections a node
+   * can have – M on upper layers, and M * 2 on the lowest level.
+   * @param beamWidth the size of the beam search to use when finding nearest 
neighbors.
+   */
+  public ConcurrentHnswGraphBuilder(
+  RandomAccessVectorValues vectorValues,
+  VectorEncoding vectorEncoding,
+  VectorSimilarityFunction similarityFunction,
+  int M,
+  int beamWidth) {
+this.vectors = createThreadSafeVectors(vectorValues);
+this.vectorsCopy = createThreadSafeVectors(vectorV

[GitHub] [lucene] jbellis commented on a diff in pull request #12421: Concurrent hnsw graph and builder, take two

2023-08-10 Thread via GitHub


jbellis commented on code in PR #12421:
URL: https://github.com/apache/lucene/pull/12421#discussion_r1290452412


##
lucene/core/src/java/org/apache/lucene/util/hnsw/ConcurrentHnswGraphBuilder.java:
##
@@ -0,0 +1,468 @@
+/*
+ * 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 static java.lang.Math.log;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.util.Objects;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Supplier;
+import org.apache.lucene.index.VectorEncoding;
+import org.apache.lucene.index.VectorSimilarityFunction;
+import org.apache.lucene.util.GrowableBitSet;
+import org.apache.lucene.util.InfoStream;
+import org.apache.lucene.util.ThreadInterruptedException;
+import org.apache.lucene.util.hnsw.ConcurrentNeighborSet.NeighborSimilarity;
+import org.apache.lucene.util.hnsw.ConcurrentOnHeapHnswGraph.NodeAtLevel;
+
+/**
+ * Builder for Concurrent HNSW graph. See {@link HnswGraph} for a high level 
overview, and the
+ * comments to `addGraphNode` for details on the concurrent building approach.
+ *
+ * @param  the type of vector
+ */
+public class ConcurrentHnswGraphBuilder {
+
+  /** Default number of maximum connections per node */
+  public static final int DEFAULT_MAX_CONN = 16;
+
+  /**
+   * Default number of the size of the queue maintained while searching during 
a graph construction.
+   */
+  public static final int DEFAULT_BEAM_WIDTH = 100;
+
+  /** A name for the HNSW component for the info-stream */
+  public static final String HNSW_COMPONENT = "HNSW";
+
+  private final int beamWidth;
+  private final double ml;
+  private final ExplicitThreadLocal scratchNeighbors;
+
+  private final VectorSimilarityFunction similarityFunction;
+  private final VectorEncoding vectorEncoding;
+  private final ExplicitThreadLocal> vectors;
+  private final ExplicitThreadLocal> graphSearcher;
+  private final ExplicitThreadLocal beamCandidates;
+
+  final ConcurrentOnHeapHnswGraph hnsw;
+  private final ConcurrentSkipListSet insertionsInProgress =
+  new ConcurrentSkipListSet<>();
+
+  private InfoStream infoStream = InfoStream.getDefault();
+
+  // we need two sources of vectors in order to perform diversity check 
comparisons without
+  // colliding
+  private final ExplicitThreadLocal> vectorsCopy;
+
+  /** This is the "native" factory for ConcurrentHnswGraphBuilder. */
+  public static  ConcurrentHnswGraphBuilder create(
+  RandomAccessVectorValues vectors,
+  VectorEncoding vectorEncoding,
+  VectorSimilarityFunction similarityFunction,
+  int M,
+  int beamWidth)
+  throws IOException {
+return new ConcurrentHnswGraphBuilder<>(
+vectors, vectorEncoding, similarityFunction, M, beamWidth);
+  }
+
+  /**
+   * Reads all the vectors from vector values, builds a graph connecting them 
by their dense
+   * ordinals, using the given hyperparameter settings, and returns the 
resulting graph.
+   *
+   * @param vectorValues the vectors whose relations are represented by the 
graph - must provide a
+   * different view over those vectors than the one used to add via 
addGraphNode.
+   * @param M – graph fanout parameter used to calculate the maximum number of 
connections a node
+   * can have – M on upper layers, and M * 2 on the lowest level.
+   * @param beamWidth the size of the beam search to use when finding nearest 
neighbors.
+   */
+  public ConcurrentHnswGraphBuilder(
+  RandomAccessVectorValues vectorValues,
+  VectorEncoding vectorEncoding,
+  VectorSimilarityFunction similarityFunction,
+  int M,
+  int beamWidth) {
+this.vectors = createThreadSafeVectors(vectorValues);
+this.vectorsCopy = createThreadSafeVectors(vectorV

[GitHub] [lucene] benwtrent opened a new pull request, #12500: Fix flaky testToString method for Knn Vector queries

2023-08-10 Thread via GitHub


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

   Periodically, the random indexer will force merge on close, this means that 
what was originally indexed as the zeroth document could no longer be the 
zeroth document.
   
   This commit adjusts the assertion to ensure the to string format is as 
expected for `DocAndScoreQuery`, regardless of the matching doc-id in the test.
   
   This seed shows the issue:
   ```
   ./gradlew test --tests TestKnnByteVectorQuery.testToString 
-Dtests.seed=B78CDB966F4B8FC5
   ```


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



[GitHub] [lucene-solr] jainankitk closed pull request #601: Adding reader settings for moving fst offheap

2023-08-10 Thread via GitHub


jainankitk closed pull request #601: Adding reader settings for moving fst 
offheap
URL: https://github.com/apache/lucene-solr/pull/601


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