You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by "benwtrent (via GitHub)" <gi...@apache.org> on 2023/07/11 21:28:03 UTC

[GitHub] [lucene] benwtrent opened a new pull request, #12434: Add ParentJoin KNN support

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

   A `join` within Lucene is built by adding child-docs and parent-docs in order. Since our vector field already supports sparse indexing, it should be able to support parent join indexing. 
   
   However, when searching for the closest `k`, it is still the k nearest children vectors with no way to join back to the parent.
   
   This commit adds this ability through some significant changes:
    - New leaf reader function that allows a collector for knn results
    - The knn results can then utilize bit-sets to join back to the parent id
    
   This change is fairly large and there are some dragons I am not sure about. So, opening as a draft for deeper discussion.
   
   FYI, I did some testing:
    - Indexing time is pretty much unaffected when doing sparse indexing
    - Search time with the parent join adds about 20% additional overhead as the change currently is. I suspect it has to do with resolving ordinals and deduplicating/ordering the hash map


-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274825776


##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected int visitedCount;
+  private final int visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, int visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  public final boolean incomplete() {
+    return visitedCount >= visitLimit;
+  }
+
+  public final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited count
+   */
+  public final int visitedCount() {
+    return visitedCount;
+  }
+
+  public final int visitLimit() {
+    return visitLimit;
+  }
+
+  public final int k() {

Review Comment:
   I'm not seeing where it's used to pre-allocate candidate collection? (To be clear, I'm only suggesting we remove it from the public API, `TopKnnResults` would still take a `k` parameter in its ctor)



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274938033


##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected long visitedCount;

Review Comment:
   It should be protected as these collectors can be used during graph building and are reused between nodes being indexed. Another option is making `earlyTerminated` overridable so that the `GraphBuilderKnnResults` can override it.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1272681200


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResultsProvider.java:
##########
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+/** knn results provider */
+public interface KnnResultsProvider {
+  KnnResults getKnnResults(IntToIntFunction vectorToOrd);

Review Comment:
   Latest commit eagerly makes the ordinal translation eager.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1273607520


##########
lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsReader.java:
##########
@@ -117,6 +118,71 @@ public abstract TopDocs search(
    */
   public abstract TopDocs search(
       String field, byte[] target, int k, Bits acceptDocs, int visitedLimit) throws IOException;
+
+  /**
+   * Return the k nearest neighbor documents as determined by comparison of their vector values for
+   * this field, to the given vector, by the field's similarity function. The score of each document
+   * is derived from the vector similarity in a way that ensures scores are positive and that a
+   * larger score corresponds to a higher ranking.
+   *
+   * <p>The search is allowed to be approximate, meaning the results are not guaranteed to be the
+   * true k closest neighbors. For large values of k (for example when k is close to the total
+   * number of documents), the search may also retrieve fewer than k documents.
+   *
+   * <p>The returned {@link TopDocs} will contain a {@link ScoreDoc} for each nearest neighbor, in
+   * order of their similarity to the query vector (decreasing scores). The {@link TotalHits}
+   * contains the number of documents visited during the search. If the search stopped early because
+   * it hit {@code visitedLimit}, it is indicated through the relation {@code
+   * TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO}.
+   *
+   * <p>The behavior is undefined if the given field doesn't have KNN vectors enabled on its {@link
+   * FieldInfo}. The return value is never {@code null}.
+   *
+   * @param field the vector field to search
+   * @param target the vector-valued query
+   * @param knnResults a KnnResults collector and relevant settings for gathering vector results
+   * @param acceptDocs {@link Bits} that represents the allowed documents to match, or {@code null}
+   *     if they are all allowed to match.
+   * @return the k nearest neighbor documents, along with their (similarity-specific) scores.
+   */
+  public TopDocs search(String field, float[] target, KnnResults knnResults, Bits acceptDocs)
+      throws IOException {
+    throw new UnsupportedOperationException(
+        "vector reader doesn't provide KNN search with results provider");

Review Comment:
   How feasible do you think it would be to only have this method on `KnnVectorsReader`, fix older codecs to implement it, and make `LeafReader#searchNearestVectors(String, float[], int, Bits, int)` final by delegating to `LeafReader#searchNearestVectors(String, float[], KnnResults, int)`?



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResults.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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 org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  /** KnnResults when exiting search early and returning empty top docs */
+  static class EmptyKnnResults extends KnnResults {
+    public EmptyKnnResults(int k, int visitedCount, int visitLimit) {
+      super(k, visitLimit);
+      this.visitedCount = visitedCount;
+    }
+
+    @Override
+    public void doClear() {}
+
+    @Override
+    public boolean collect(int vectorId, float similarity) {
+      throw new IllegalArgumentException();
+    }
+
+    @Override
+    public boolean isFull() {
+      return true;
+    }
+
+    @Override
+    public float minSimilarity() {
+      return 0;
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TotalHits th = new TotalHits(visitedCount, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO);
+      return new TopDocs(th, new ScoreDoc[0]);
+    }
+  }
+
+  static class OrdinalTranslatedKnnResults extends KnnResults {
+    private final KnnResults in;
+    private final IntToIntFunction vectorOrdinalToDocId;
+
+    OrdinalTranslatedKnnResults(KnnResults in, IntToIntFunction vectorOrdinalToDocId) {
+      super(in.k, in.visitLimit);
+      this.in = in;
+      this.vectorOrdinalToDocId = vectorOrdinalToDocId;
+    }
+
+    @Override
+    void doClear() {
+      in.clear();
+    }
+
+    @Override
+    boolean collect(int vectorId, float similarity) {
+      return in.collect(vectorOrdinalToDocId.apply(vectorId), similarity);
+    }
+
+    @Override
+    boolean isFull() {
+      return in.isFull();
+    }
+
+    @Override
+    float minSimilarity() {
+      return in.minSimilarity();
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TopDocs td = in.topDocs();
+      return new TopDocs(
+          new TotalHits(
+              visitedCount(),
+              incomplete()
+                  ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+                  : TotalHits.Relation.EQUAL_TO),
+          td.scoreDocs);
+    }
+  }
+
+  protected int visitedCount;
+  private final int visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, int visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  final void clear() {
+    this.visitedCount = 0;
+    doClear();
+  }
+
+  /** Clear the current results. */
+  abstract void doClear();

Review Comment:
   Both `clear` and `doClear` seem to be only needed for graph building, could we remove them from here?



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/ToParentJoinKnnResults.java:
##########
@@ -0,0 +1,303 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BitSet;
+
+/** parent joining knn results, vectorIds are deduplicated according to the parent bit set. */
+public class ToParentJoinKnnResults extends KnnResults {

Review Comment:
   Could this live in the lucene/join module instead of core? Likewise for `NodeIdCachingHead`?



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResults.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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 org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {

Review Comment:
   This class should probably be in oal.search rather than oal.util.hnsw since it's exposed by the search APIs (but the empty and ordinals-translating impls should stay here since they're implementation details of HNSWGraphBuilder)?



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResults.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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 org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  /** KnnResults when exiting search early and returning empty top docs */
+  static class EmptyKnnResults extends KnnResults {
+    public EmptyKnnResults(int k, int visitedCount, int visitLimit) {
+      super(k, visitLimit);
+      this.visitedCount = visitedCount;
+    }
+
+    @Override
+    public void doClear() {}
+
+    @Override
+    public boolean collect(int vectorId, float similarity) {
+      throw new IllegalArgumentException();
+    }
+
+    @Override
+    public boolean isFull() {
+      return true;
+    }
+
+    @Override
+    public float minSimilarity() {
+      return 0;
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TotalHits th = new TotalHits(visitedCount, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO);
+      return new TopDocs(th, new ScoreDoc[0]);
+    }
+  }
+
+  static class OrdinalTranslatedKnnResults extends KnnResults {
+    private final KnnResults in;
+    private final IntToIntFunction vectorOrdinalToDocId;
+
+    OrdinalTranslatedKnnResults(KnnResults in, IntToIntFunction vectorOrdinalToDocId) {
+      super(in.k, in.visitLimit);
+      this.in = in;
+      this.vectorOrdinalToDocId = vectorOrdinalToDocId;
+    }
+
+    @Override
+    void doClear() {
+      in.clear();
+    }
+
+    @Override
+    boolean collect(int vectorId, float similarity) {
+      return in.collect(vectorOrdinalToDocId.apply(vectorId), similarity);
+    }
+
+    @Override
+    boolean isFull() {
+      return in.isFull();
+    }
+
+    @Override
+    float minSimilarity() {
+      return in.minSimilarity();
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TopDocs td = in.topDocs();
+      return new TopDocs(
+          new TotalHits(
+              visitedCount(),
+              incomplete()
+                  ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+                  : TotalHits.Relation.EQUAL_TO),
+          td.scoreDocs);
+    }
+  }
+
+  protected int visitedCount;
+  private final int visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, int visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  final void clear() {
+    this.visitedCount = 0;
+    doClear();
+  }
+
+  /** Clear the current results. */
+  abstract void doClear();
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  final boolean incomplete() {
+    return visitedCount >= visitLimit;
+  }
+
+  final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited count
+   */
+  final int visitedCount() {
+    return visitedCount;
+  }
+
+  final int visitLimit() {
+    return visitLimit;
+  }
+
+  public final int k() {
+    return k;
+  }
+
+  /**
+   * Collect the provided vectorId and include in the result set.
+   *
+   * @param vectorId the vector to collect
+   * @param similarity its calculated similarity
+   * @return true if the vector is collected
+   */
+  abstract boolean collect(int vectorId, float similarity);

Review Comment:
   The first parameter should be called docID now, right?



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResults.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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 org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  /** KnnResults when exiting search early and returning empty top docs */
+  static class EmptyKnnResults extends KnnResults {
+    public EmptyKnnResults(int k, int visitedCount, int visitLimit) {
+      super(k, visitLimit);
+      this.visitedCount = visitedCount;
+    }
+
+    @Override
+    public void doClear() {}
+
+    @Override
+    public boolean collect(int vectorId, float similarity) {
+      throw new IllegalArgumentException();
+    }
+
+    @Override
+    public boolean isFull() {
+      return true;
+    }
+
+    @Override
+    public float minSimilarity() {
+      return 0;
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TotalHits th = new TotalHits(visitedCount, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO);
+      return new TopDocs(th, new ScoreDoc[0]);
+    }
+  }
+
+  static class OrdinalTranslatedKnnResults extends KnnResults {
+    private final KnnResults in;
+    private final IntToIntFunction vectorOrdinalToDocId;
+
+    OrdinalTranslatedKnnResults(KnnResults in, IntToIntFunction vectorOrdinalToDocId) {
+      super(in.k, in.visitLimit);
+      this.in = in;
+      this.vectorOrdinalToDocId = vectorOrdinalToDocId;
+    }
+
+    @Override
+    void doClear() {
+      in.clear();
+    }
+
+    @Override
+    boolean collect(int vectorId, float similarity) {
+      return in.collect(vectorOrdinalToDocId.apply(vectorId), similarity);
+    }
+
+    @Override
+    boolean isFull() {
+      return in.isFull();
+    }
+
+    @Override
+    float minSimilarity() {
+      return in.minSimilarity();
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TopDocs td = in.topDocs();
+      return new TopDocs(
+          new TotalHits(
+              visitedCount(),
+              incomplete()
+                  ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+                  : TotalHits.Relation.EQUAL_TO),
+          td.scoreDocs);
+    }
+  }
+
+  protected int visitedCount;
+  private final int visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, int visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  final void clear() {
+    this.visitedCount = 0;
+    doClear();
+  }
+
+  /** Clear the current results. */
+  abstract void doClear();
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  final boolean incomplete() {
+    return visitedCount >= visitLimit;
+  }
+
+  final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited count
+   */
+  final int visitedCount() {
+    return visitedCount;
+  }
+
+  final int visitLimit() {
+    return visitLimit;
+  }
+
+  public final int k() {
+    return k;
+  }
+
+  /**
+   * Collect the provided vectorId and include in the result set.
+   *
+   * @param vectorId the vector to collect
+   * @param similarity its calculated similarity
+   * @return true if the vector is collected
+   */
+  abstract boolean collect(int vectorId, float similarity);
+
+  /**
+   * @return Is the current result set considered full
+   */
+  abstract boolean isFull();

Review Comment:
   I wonder if we can avoid introducing this method and change call-sites to check if minSimilarity() returns a greater value than `NEGATIVE_INFINITY`, in order to keep the surface area minimal.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1273984142


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResults.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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 org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  /** KnnResults when exiting search early and returning empty top docs */
+  static class EmptyKnnResults extends KnnResults {
+    public EmptyKnnResults(int k, int visitedCount, int visitLimit) {
+      super(k, visitLimit);
+      this.visitedCount = visitedCount;
+    }
+
+    @Override
+    public void doClear() {}
+
+    @Override
+    public boolean collect(int vectorId, float similarity) {
+      throw new IllegalArgumentException();
+    }
+
+    @Override
+    public boolean isFull() {
+      return true;
+    }
+
+    @Override
+    public float minSimilarity() {
+      return 0;
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TotalHits th = new TotalHits(visitedCount, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO);
+      return new TopDocs(th, new ScoreDoc[0]);
+    }
+  }
+
+  static class OrdinalTranslatedKnnResults extends KnnResults {
+    private final KnnResults in;
+    private final IntToIntFunction vectorOrdinalToDocId;
+
+    OrdinalTranslatedKnnResults(KnnResults in, IntToIntFunction vectorOrdinalToDocId) {
+      super(in.k, in.visitLimit);
+      this.in = in;
+      this.vectorOrdinalToDocId = vectorOrdinalToDocId;
+    }
+
+    @Override
+    void doClear() {
+      in.clear();
+    }
+
+    @Override
+    boolean collect(int vectorId, float similarity) {
+      return in.collect(vectorOrdinalToDocId.apply(vectorId), similarity);
+    }
+
+    @Override
+    boolean isFull() {
+      return in.isFull();
+    }
+
+    @Override
+    float minSimilarity() {
+      return in.minSimilarity();
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TopDocs td = in.topDocs();
+      return new TopDocs(
+          new TotalHits(
+              visitedCount(),
+              incomplete()
+                  ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+                  : TotalHits.Relation.EQUAL_TO),
+          td.scoreDocs);
+    }
+  }
+
+  protected int visitedCount;
+  private final int visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, int visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  final void clear() {
+    this.visitedCount = 0;
+    doClear();
+  }
+
+  /** Clear the current results. */
+  abstract void doClear();
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  final boolean incomplete() {
+    return visitedCount >= visitLimit;
+  }
+
+  final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited count
+   */
+  final int visitedCount() {
+    return visitedCount;
+  }
+
+  final int visitLimit() {
+    return visitLimit;
+  }
+
+  public final int k() {
+    return k;
+  }
+
+  /**
+   * Collect the provided vectorId and include in the result set.
+   *
+   * @param vectorId the vector to collect
+   * @param similarity its calculated similarity
+   * @return true if the vector is collected
+   */
+  abstract boolean collect(int vectorId, float similarity);

Review Comment:
   From the perspective of the user, it is always a doc ID it seems, ie if they pass a KnnResults object to the search method, it will always see doc IDs?



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1638442919

   @jpountz my original benchmarks were flawed. There was a bug in my testing. Nested is actually 80% slower (or 1.8x times) than the current search times. 
   
   I am investigating the current possible causes.


-- 
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-unsubscribe@lucene.apache.org

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


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


Re: [PR] Add ParentJoin KNN support [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1836025921

   @david-sitsky sorry for the confusion, it was renamed `DiversifyingChildren*KnnVectorQuery`


-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1276485454


##########
lucene/core/src/java/org/apache/lucene/search/TopKnnCollector.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.search;
+
+import org.apache.lucene.util.hnsw.NeighborQueue;
+
+/**
+ * TopKnnCollector is a specific KnnCollector. A minHeap is used to keep track of the currently
+ * collected vectors allowing for efficient updates as better vectors are collected.
+ *
+ * @lucene.experimental
+ */
+public final class TopKnnCollector extends AbstractKnnCollector {
+
+  private final NeighborQueue queue;
+
+  /**
+   * @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) {
+    super(k, visitLimit);
+    this.queue = new NeighborQueue(k, false);
+  }
+
+  @Override
+  public boolean collect(int docId, float similarity) {
+    return queue.insertWithOverflow(docId, similarity);
+  }
+
+  @Override
+  public float minCompetitiveSimilarity() {
+    return queue.size() >= k() ? queue.topScore() : Float.NEGATIVE_INFINITY;
+  }
+
+  @Override
+  public TopDocs topDocs() {
+    while (queue.size() > k()) {

Review Comment:
   Can it actually happen? It looks like we always add to the queue via `insertWithOverflow`, which should guarantee that there are never more than k entries in it?



##########
lucene/core/src/java/org/apache/lucene/search/TopKnnCollector.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.search;
+
+import org.apache.lucene.util.hnsw.NeighborQueue;
+
+/**
+ * TopKnnCollector is a specific KnnCollector. A minHeap is used to keep track of the currently
+ * collected vectors allowing for efficient updates as better vectors are collected.
+ *
+ * @lucene.experimental
+ */
+public final class TopKnnCollector extends AbstractKnnCollector {
+
+  private final NeighborQueue queue;
+
+  /**
+   * @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) {
+    super(k, visitLimit);
+    this.queue = new NeighborQueue(k, false);
+  }
+
+  @Override
+  public boolean collect(int docId, float similarity) {
+    return queue.insertWithOverflow(docId, similarity);
+  }
+
+  @Override
+  public float minCompetitiveSimilarity() {
+    return queue.size() >= k() ? queue.topScore() : Float.NEGATIVE_INFINITY;
+  }
+
+  @Override
+  public TopDocs topDocs() {
+    while (queue.size() > k()) {
+      queue.pop();
+    }
+    int i = 0;
+    ScoreDoc[] scoreDocs = new ScoreDoc[queue.size()];
+    while (i < scoreDocs.length) {

Review Comment:
   nit: this loop would be simpler as a regular for loop?



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274014277


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResults.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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 org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  /** KnnResults when exiting search early and returning empty top docs */
+  static class EmptyKnnResults extends KnnResults {
+    public EmptyKnnResults(int k, int visitedCount, int visitLimit) {
+      super(k, visitLimit);
+      this.visitedCount = visitedCount;
+    }
+
+    @Override
+    public void doClear() {}
+
+    @Override
+    public boolean collect(int vectorId, float similarity) {
+      throw new IllegalArgumentException();
+    }
+
+    @Override
+    public boolean isFull() {
+      return true;
+    }
+
+    @Override
+    public float minSimilarity() {
+      return 0;
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TotalHits th = new TotalHits(visitedCount, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO);
+      return new TopDocs(th, new ScoreDoc[0]);
+    }
+  }
+
+  static class OrdinalTranslatedKnnResults extends KnnResults {
+    private final KnnResults in;
+    private final IntToIntFunction vectorOrdinalToDocId;
+
+    OrdinalTranslatedKnnResults(KnnResults in, IntToIntFunction vectorOrdinalToDocId) {
+      super(in.k, in.visitLimit);
+      this.in = in;
+      this.vectorOrdinalToDocId = vectorOrdinalToDocId;
+    }
+
+    @Override
+    void doClear() {
+      in.clear();
+    }
+
+    @Override
+    boolean collect(int vectorId, float similarity) {
+      return in.collect(vectorOrdinalToDocId.apply(vectorId), similarity);
+    }
+
+    @Override
+    boolean isFull() {
+      return in.isFull();
+    }
+
+    @Override
+    float minSimilarity() {
+      return in.minSimilarity();
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TopDocs td = in.topDocs();
+      return new TopDocs(
+          new TotalHits(
+              visitedCount(),
+              incomplete()
+                  ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+                  : TotalHits.Relation.EQUAL_TO),
+          td.scoreDocs);
+    }
+  }
+
+  protected int visitedCount;
+  private final int visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, int visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  final void clear() {
+    this.visitedCount = 0;
+    doClear();
+  }
+
+  /** Clear the current results. */
+  abstract void doClear();
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  final boolean incomplete() {
+    return visitedCount >= visitLimit;
+  }
+
+  final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited count
+   */
+  final int visitedCount() {
+    return visitedCount;
+  }
+
+  final int visitLimit() {
+    return visitLimit;
+  }
+
+  public final int k() {
+    return k;
+  }
+
+  /**
+   * Collect the provided vectorId and include in the result set.
+   *
+   * @param vectorId the vector to collect
+   * @param similarity its calculated similarity
+   * @return true if the vector is collected
+   */
+  abstract boolean collect(int vectorId, float similarity);

Review Comment:
   I can see that



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274948941


##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected long visitedCount;

Review Comment:
   If I make `KnnResults` an interface and make `GraphBuilderKnnResults` implement it instead, I could just ignore many of the methods. Then this visited count could be private. I will refactor.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] msokolov commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "msokolov (via GitHub)" <gi...@apache.org>.
msokolov commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1279451735


##########
lucene/core/src/java/org/apache/lucene/search/KnnCollector.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.search;
+
+/**
+ * KnnCollector is a knn collector used for gathering kNN results and providing topDocs from the
+ * gathered neighbors
+ *
+ * @lucene.experimental
+ */
+public interface KnnCollector {
+
+  /**
+   * If search visits too many documents, the results collector will terminate early. Usually, this
+   * is due to some restricted filter on the document set.
+   *
+   * <p>When collection is earlyTerminated, the results are not a correct representation of k
+   * nearest neighbors.
+   *
+   * @return is the current result set marked as incomplete?
+   */
+  boolean earlyTerminated();
+
+  /**
+   * @param count increments the visited vector count, must be greater than 0.
+   */
+  void incVisitedCount(int count);
+
+  /**
+   * @return the current visited vector count
+   */
+  long visitedCount();
+
+  /**
+   * @return the visited vector limit
+   */
+  long visitLimit();
+
+  /**
+   * @return the expected number of collected results
+   */
+  int k();
+
+  /**
+   * Collect the provided docId and include in the result set.

Review Comment:
   I think the implementation is fine; it's just the javadoc that bothers me, because KnnCollectors (such as OrdinalTranslatedKnnCollector) may be passed vectorIds. But if that's really the only one, and it's internal, then I guess this is 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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1644019338

   Here are two more JFRs, one with nested count of 128, the other with 512.
   
   From what I can tell, the majority of the time is spent correcting the vector ordinal & getting the parent doc id. The caching stack overhead seems much smaller than that overhead.
   
   [many-nested.zip](https://github.com/apache/lucene/files/12110287/many-nested.zip)
   
   


-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1269777197


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResultsProvider.java:
##########
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+/** knn results provider */
+public interface KnnResultsProvider {

Review Comment:
   @jpountz my concern is that we will be constructing `KnnResults` object (and allocating storage arrays) even if it isn't used (e.g. we exit early). 
   
   I guess by using a `"provider"` here I am optimizing for an exceptional case :/ I can switch it.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274813992


##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected int visitedCount;
+  private final int visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, int visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  public final boolean incomplete() {

Review Comment:
   Or `earlyTerminated`? 



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1275147272


##########
lucene/core/src/java/org/apache/lucene/search/KnnFloatVectorQuery.java:
##########
@@ -78,8 +79,16 @@ public KnnFloatVectorQuery(String field, float[] target, int k, Query filter) {
   @Override
   protected TopDocs approximateSearch(LeafReaderContext context, Bits acceptDocs, int visitedLimit)
       throws IOException {
+    FieldInfo fi = context.reader().getFieldInfos().fieldInfo(field);
+    if (fi == null || fi.getVectorDimension() == 0) {
+      // The field does not exist or does not index vectors
+      return NO_RESULTS;
+    }
+    int k = Math.min(this.k, context.reader().getFloatVectorValues(fi.name).size());
     TopDocs results =
-        context.reader().searchNearestVectors(field, target, k, acceptDocs, visitedLimit);
+        context
+            .reader()
+            .searchNearestVectors(field, target, new TopKnnCollector(k, visitedLimit), acceptDocs);

Review Comment:
   like on the byte vector query, we could push this logic of checking for field existence and reducing `k` to `LeafReader`?



##########
lucene/core/src/java/org/apache/lucene/search/KnnCollector.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.search;
+
+/**
+ * KnnCollector is a knn collector used for gathering kNN results and providing topDocs from the
+ * gathered neighbors
+ */
+public interface KnnCollector {
+
+  /**
+   * If search visits too many documents, the results collector will terminate early. Usually, this
+   * is due to some restricted filter on the document set.
+   *
+   * <p>When collection is earlyTerminated, the results are not a correct representation of k
+   * nearest neighbors.
+   *
+   * @return is the current result set marked as incomplete?
+   */
+  boolean earlyTerminated();
+
+  /**
+   * @param count increments the visited vector count, must be greater than 0.
+   */
+  void incVisitedCount(int count);
+
+  /**
+   * @return the current visited vector count
+   */
+  long visitedCount();
+
+  /**
+   * @return the visited vector limit
+   */
+  long visitLimit();
+
+  /**
+   * @return the expected number of collected results
+   */
+  int k();
+
+  /**
+   * Collect the provided docId and include in the result set.
+   *
+   * @param docId of the vector to collect
+   * @param similarity its calculated similarity
+   * @return true if the vector is collected
+   */
+  boolean collect(int docId, float similarity);
+
+  /**
+   * This method is utilized during search to ensure only competitive results are explored.
+   *
+   * <p>Consequently, if this results collector wants to collect `k` results, this should return
+   * {@link Float#NEGATIVE_INFINITY} when not full.
+   *
+   * <p>When full, the minimum score should be returned.
+   *
+   * @return the current minimum competitive similarity in the collection
+   */
+  float minCompetitiveSimilarity();
+
+  /**
+   * This drains the collected nearest kNN results and returns them in a new {@link TopDocs}
+   * collection, ordered by score descending

Review Comment:
   "drains" suggests it already, but for extra clarity, maybe mention that this is a destructive operation



##########
lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsReader.java:
##########
@@ -79,14 +80,13 @@ protected KnnVectorsReader() {}
    *
    * @param field the vector field to search
    * @param target the vector-valued query
-   * @param k the number of docs to return
+   * @param knnResults a KnnResults collector and relevant settings for gathering vector results
    * @param acceptDocs {@link Bits} that represents the allowed documents to match, or {@code null}
    *     if they are all allowed to match.
-   * @param visitedLimit the maximum number of nodes that the search is allowed to visit
    * @return the k nearest neighbor documents, along with their (similarity-specific) scores.
    */
   public abstract TopDocs search(
-      String field, float[] target, int k, Bits acceptDocs, int visitedLimit) throws IOException;
+      String field, float[] target, KnnResults knnResults, Bits acceptDocs) throws IOException;

Review Comment:
   Not a strong opinion, but when there is a destructive operation like `KnnResults#topDocs`, I prefer that we make the same component responsible for creating the object and calling the destructive method, while here, the user would be responsible for creating the `KnnResults` object and then the codec is responsible for calling `KnnResults#topDocs`. If it returned void, the caller would be responsible for both creating the object and calling the destructive method.



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/TopKnnCollector.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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 org.apache.lucene.search.AbstractKnnCollector;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+
+/**
+ * TopKnnCollector is a specific KnnResults. A minHeap is used to keep track of the currently

Review Comment:
   ```suggestion
    * TopKnnCollector is a specific KnnCollector. A minHeap is used to keep track of the currently
   ```



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/TopKnnCollector.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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;

Review Comment:
   It should be in oal.search I believe as it's not specific to our hnsw implementation?



##########
lucene/core/src/java/org/apache/lucene/search/KnnByteVectorQuery.java:
##########
@@ -77,8 +78,16 @@ public KnnByteVectorQuery(String field, byte[] target, int k, Query filter) {
   @Override
   protected TopDocs approximateSearch(LeafReaderContext context, Bits acceptDocs, int visitedLimit)
       throws IOException {
+    FieldInfo fi = context.reader().getFieldInfos().fieldInfo(field);
+    if (fi == null || fi.getVectorDimension() == 0) {
+      // The field does not exist or does not index vectors
+      return NO_RESULTS;
+    }
+    int k = Math.min(this.k, context.reader().getByteVectorValues(fi.name).size());

Review Comment:
   Should we have this logic in the  `LeafReader#searchNearestVectors` that takes an `int k`, and call it here instead of the expert method that takes a `KnnCollector`?



##########
lucene/core/src/java/org/apache/lucene/search/AbstractKnnCollector.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.search;
+
+/**
+ * AbstractKnnCollector is the default implementation for a knn collector used for gathering kNN
+ * results and providing topDocs from the gathered neighbors
+ */
+public abstract class AbstractKnnCollector implements KnnCollector {
+
+  private long visitedCount;
+  private final long visitLimit;
+  private final int k;
+
+  protected AbstractKnnCollector(int k, long visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  /**
+   * If search visits too many documents, the results collector will terminate early. Usually, this
+   * is due to some restricted filter on the document set.
+   *
+   * <p>When collection is earlyTerminated, the results are not a correct representation of k
+   * nearest neighbors.
+   *
+   * @return is the current result set marked as incomplete?
+   */
+  @Override
+  public final boolean earlyTerminated() {
+    return visitedCount >= visitLimit;
+  }
+
+  /**
+   * @param count increments the visited vector count, must be greater than 0.
+   */
+  @Override
+  public final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited vector count
+   */
+  @Override
+  public final long visitedCount() {
+    return visitedCount;
+  }
+
+  /**
+   * @return the visited vector limit
+   */
+  @Override
+  public final long visitLimit() {
+    return visitLimit;
+  }
+
+  /**
+   * @return the expected number of collected results
+   */
+  @Override
+  public final int k() {
+    return k;
+  }
+
+  /**
+   * Collect the provided docId and include in the result set.
+   *
+   * @param docId of the vector to collect
+   * @param similarity its calculated similarity
+   * @return true if the vector is collected
+   */
+  @Override
+  public abstract boolean collect(int docId, float similarity);
+
+  /**
+   * This method is utilized during search to ensure only competitive results are explored.
+   *
+   * <p>Consequently, if this results collector wants to collect `k` results, this should return
+   * {@link Float#NEGATIVE_INFINITY} when not full.
+   *
+   * <p>When full, the minimum score should be returned.
+   *
+   * @return the current minimum competitive similarity in the collection
+   */
+  @Override
+  public abstract float minCompetitiveSimilarity();
+
+  /**
+   * This drains the collected nearest kNN results and returns them in a new {@link TopDocs}
+   * collection, ordered by score descending
+   *
+   * @return The collected top documents
+   */

Review Comment:
   We don't need these docs here as they're already on the interface?



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1269632250


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResults.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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 org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  /** KnnResults when exiting search early and returning empty top docs */
+  static class EmptyKnnResults extends KnnResults {
+    public EmptyKnnResults(int visitedCount) {
+      this.visitedCount = visitedCount;
+    }
+
+    @Override
+    public void doClear() {}
+
+    @Override
+    public void collect(int vectorId, float similarity) {
+      throw new IllegalArgumentException();
+    }
+
+    @Override
+    public boolean collectWithOverflow(int vectorId, float similarity) {
+      return false;
+    }
+
+    @Override
+    public boolean isFull() {
+      return true;
+    }
+
+    @Override
+    public float minSimilarity() {
+      return 0;
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TotalHits th = new TotalHits(visitedCount, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO);
+      return new TopDocs(th, new ScoreDoc[0]);
+    }
+  }
+
+  protected int visitedCount;
+  private boolean incomplete;
+
+  final void clear() {
+    this.visitedCount = 0;
+    this.incomplete = false;
+    doClear();
+  }
+
+  /** Clear the current results. */
+  abstract void doClear();
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  final boolean incomplete() {
+    return incomplete;
+  }
+
+  /** Mark the current result set as incomplete */
+  final void markIncomplete() {
+    this.incomplete = true;
+  }
+
+  /**
+   * @param count set the current visited count to the provided value
+   */
+  final void setVisitedCount(int count) {
+    this.visitedCount = count;
+  }
+
+  /**
+   * @return the current visited count
+   */
+  final int visitedCount() {
+    return visitedCount;
+  }
+
+  /**
+   * Collect the provided vectorId and include in the result set.
+   *
+   * @param vectorId the vector to collect
+   * @param similarity its calculated similarity
+   */
+  abstract void collect(int vectorId, float similarity);
+
+  /**
+   * @param vectorId the vector to collect
+   * @param similarity its calculated similarity
+   * @return true if the vector is collected
+   */
+  abstract boolean collectWithOverflow(int vectorId, float similarity);

Review Comment:
   we seem to be doing a `collectWithOverflow` when the result set is full, so maybe we could have a single `collect` method and automatically do the right thing depending on whether it's full or not?



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResultsProvider.java:
##########
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+/** knn results provider */
+public interface KnnResultsProvider {
+  KnnResults getKnnResults(IntToIntFunction vectorToOrd);

Review Comment:
   This vectorToOrd argument feels specific to the current implementation. Could the API work on doc IDs? 



##########
lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsReader.java:
##########
@@ -117,6 +118,85 @@ public abstract TopDocs search(
    */
   public abstract TopDocs search(
       String field, byte[] target, int k, Bits acceptDocs, int visitedLimit) throws IOException;
+
+  /**
+   * Return the k nearest neighbor documents as determined by comparison of their vector values for
+   * this field, to the given vector, by the field's similarity function. The score of each document
+   * is derived from the vector similarity in a way that ensures scores are positive and that a
+   * larger score corresponds to a higher ranking.
+   *
+   * <p>The search is allowed to be approximate, meaning the results are not guaranteed to be the
+   * true k closest neighbors. For large values of k (for example when k is close to the total
+   * number of documents), the search may also retrieve fewer than k documents.
+   *
+   * <p>The returned {@link TopDocs} will contain a {@link ScoreDoc} for each nearest neighbor, in
+   * order of their similarity to the query vector (decreasing scores). The {@link TotalHits}
+   * contains the number of documents visited during the search. If the search stopped early because
+   * it hit {@code visitedLimit}, it is indicated through the relation {@code
+   * TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO}.
+   *
+   * <p>The behavior is undefined if the given field doesn't have KNN vectors enabled on its {@link
+   * FieldInfo}. The return value is never {@code null}.
+   *
+   * @param field the vector field to search
+   * @param target the vector-valued query
+   * @param knnResultsProvider a provider that returns a KnnResults collector and topK for gathering
+   *     the vector results
+   * @param acceptDocs {@link Bits} that represents the allowed documents to match, or {@code null}
+   *     if they are all allowed to match.
+   * @param visitedLimit the maximum number of nodes that the search is allowed to visit
+   * @return the k nearest neighbor documents, along with their (similarity-specific) scores.
+   */
+  public TopDocs search(
+      String field,
+      float[] target,
+      KnnResultsProvider knnResultsProvider,
+      Bits acceptDocs,
+      int visitedLimit)

Review Comment:
   If we're adding this more flexible way of collecting KNN hits, it would be nice if visitedLimit could be handled directly by the `KnnResultsProvider` instead of being a separate parameter. Handling `acceptDocs` via this new interface crossed my mind too though I'm less sure about it since collectors don't need to handle deleted docs and it might be nice to have consistency between collectors and this new abstraction you're adding, which looks similar.



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/ToParentJoinKnnResults.java:
##########
@@ -0,0 +1,362 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BitSet;
+
+/** parent joining knn results, vectorIds are deduplicated according to the parent bit set. */
+public class ToParentJoinKnnResults extends KnnResults {
+
+  /** provider class for creating a new {@link ToParentJoinKnnResults} */
+  public static class Provider implements KnnResultsProvider {
+
+    private final int k;
+    private final BitSet parentBitSet;
+
+    public Provider(int k, BitSet parentBitSet) {
+      this.k = k;
+      this.parentBitSet = parentBitSet;
+    }
+
+    @Override
+    public int k() {
+      return k;
+    }
+
+    @Override
+    public KnnResults getKnnResults(IntToIntFunction vectorToOrd) {
+      return new ToParentJoinKnnResults(k, parentBitSet, vectorToOrd);
+    }
+  }
+
+  private final BitSet parentBitSet;
+  private final int k;
+  private final IntToIntFunction vectorToOrd;
+  private final NodeIdCachingHeap heap;
+
+  public ToParentJoinKnnResults(int k, BitSet parentBitSet, IntToIntFunction vectorToOrd) {
+    this.parentBitSet = parentBitSet;
+    this.k = k;
+    this.vectorToOrd = vectorToOrd;
+    this.heap = new NodeIdCachingHeap(k);
+  }
+
+  /**
+   * Adds a new graph arc, extending the storage as needed.
+   *
+   * <p>If the provided childNodeId's parent has been previously collected and the nodeScore is less
+   * than the previously stored score, this node will not be added to the collection.
+   *
+   * @param childNodeId the neighbor node id
+   * @param nodeScore the score of the neighbor, relative to some other node
+   */
+  @Override
+  public void collect(int childNodeId, float nodeScore) {
+    childNodeId = vectorToOrd.apply(childNodeId);
+    assert !parentBitSet.get(childNodeId);
+    int nodeId = parentBitSet.nextSetBit(childNodeId);
+    heap.push(nodeId, nodeScore);
+  }
+
+  /**
+   * If the heap is not full (size is less than the initialSize provided to the constructor), adds a
+   * new node-and-score element. If the heap is full, compares the score against the current top
+   * score, and replaces the top element if newScore is better than (greater than unless the heap is
+   * reversed), the current top score.
+   *
+   * <p>If childNodeId's parent node has previously been collected and the provided nodeScore is
+   * less than the stored score it will not be collected.
+   *
+   * @param childNodeId the neighbor node id
+   * @param nodeScore the score of the neighbor, relative to some other node
+   */
+  @Override
+  public boolean collectWithOverflow(int childNodeId, float nodeScore) {
+    // Parent and child nodes should be disjoint sets parent bit set should never have a child node
+    // ID present
+    childNodeId = vectorToOrd.apply(childNodeId);
+    assert !parentBitSet.get(childNodeId);
+    int nodeId = parentBitSet.nextSetBit(childNodeId);
+    return heap.insertWithOverflow(nodeId, nodeScore);
+  }
+
+  @Override
+  public boolean isFull() {
+    return heap.size >= k;
+  }
+
+  @Override
+  public float minSimilarity() {
+    return heap.topScore();
+  }
+
+  @Override
+  public void doClear() {
+    heap.clear();
+  }
+
+  @Override
+  public String toString() {
+    return "ToParentJoinKnnResults[" + heap.size + "]";
+  }
+
+  @Override
+  public TopDocs topDocs() {
+    while (heap.size() > k) {
+      heap.popToDrain();
+    }
+    int i = 0;
+    ScoreDoc[] scoreDocs = new ScoreDoc[heap.size()];
+    while (i < scoreDocs.length) {
+      int node = heap.topNode();
+      float score = heap.topScore();
+      heap.popToDrain();
+      scoreDocs[scoreDocs.length - ++i] = new ScoreDoc(node, score);
+    }
+
+    TotalHits.Relation relation =
+        incomplete() ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO : TotalHits.Relation.EQUAL_TO;
+    return new TopDocs(new TotalHits(visitedCount(), relation), scoreDocs);

Review Comment:
   Using the `visitedCount()` doesn't feel correct, as it'd be counting nested docs rather than unique parent docs? It doesn't feel super useful either, maybe we should always set it to 1 (and TotalHits.RELATION.GREATER_THAN_OR_EQUAL_TO)?



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResultsProvider.java:
##########
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+/** knn results provider */
+public interface KnnResultsProvider {

Review Comment:
   I wonder if we need this indirection with KnnResultsProvider/KnnResults, or if we could just have KnnResults?



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] msokolov commented on pull request #12434: Add ParentJoin KNN support

Posted by "msokolov (via GitHub)" <gi...@apache.org>.
msokolov commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1657264032

   @jpountz I think I see what you mean re: the cost of evaluating filters. We have been working on a solution where the most common and restrictive filters are cached and can be applied as pre-filters without needing to recompute for every query, while also joining with other constraints (post-filtering). This has led to increased yield (more high-scoring matches) at acceptable cost increase in many cases but in some cases we see increased costs without much better yield (as compared to applying all the filters as post-filters). So I feel like if the parent doc filter is highly restrictive there could be a similar tradeoff in this case ie even when the cost of the parent doc filter computation is not a consideration you might still see better recall/latency tradeoff using overcollection/post-filtering. I don't think this has any implications for this change really, but I do think it would be interesting to see some evaluation comparing.


-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent merged pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent merged PR #12434:
URL: https://github.com/apache/lucene/pull/12434


-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274490843


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResults.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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 org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  /** KnnResults when exiting search early and returning empty top docs */
+  static class EmptyKnnResults extends KnnResults {
+    public EmptyKnnResults(int k, int visitedCount, int visitLimit) {
+      super(k, visitLimit);
+      this.visitedCount = visitedCount;
+    }
+
+    @Override
+    public void doClear() {}
+
+    @Override
+    public boolean collect(int vectorId, float similarity) {
+      throw new IllegalArgumentException();
+    }
+
+    @Override
+    public boolean isFull() {
+      return true;
+    }
+
+    @Override
+    public float minSimilarity() {
+      return 0;
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TotalHits th = new TotalHits(visitedCount, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO);
+      return new TopDocs(th, new ScoreDoc[0]);
+    }
+  }
+
+  static class OrdinalTranslatedKnnResults extends KnnResults {
+    private final KnnResults in;
+    private final IntToIntFunction vectorOrdinalToDocId;
+
+    OrdinalTranslatedKnnResults(KnnResults in, IntToIntFunction vectorOrdinalToDocId) {
+      super(in.k, in.visitLimit);
+      this.in = in;
+      this.vectorOrdinalToDocId = vectorOrdinalToDocId;
+    }
+
+    @Override
+    void doClear() {
+      in.clear();
+    }
+
+    @Override
+    boolean collect(int vectorId, float similarity) {
+      return in.collect(vectorOrdinalToDocId.apply(vectorId), similarity);
+    }
+
+    @Override
+    boolean isFull() {
+      return in.isFull();
+    }
+
+    @Override
+    float minSimilarity() {
+      return in.minSimilarity();
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TopDocs td = in.topDocs();
+      return new TopDocs(
+          new TotalHits(
+              visitedCount(),
+              incomplete()
+                  ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+                  : TotalHits.Relation.EQUAL_TO),
+          td.scoreDocs);
+    }
+  }
+
+  protected int visitedCount;
+  private final int visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, int visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  final void clear() {
+    this.visitedCount = 0;
+    doClear();
+  }
+
+  /** Clear the current results. */
+  abstract void doClear();
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  final boolean incomplete() {
+    return visitedCount >= visitLimit;
+  }
+
+  final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited count
+   */
+  final int visitedCount() {
+    return visitedCount;
+  }
+
+  final int visitLimit() {
+    return visitLimit;
+  }
+
+  public final int k() {
+    return k;
+  }
+
+  /**
+   * Collect the provided vectorId and include in the result set.
+   *
+   * @param vectorId the vector to collect
+   * @param similarity its calculated similarity
+   * @return true if the vector is collected
+   */
+  abstract boolean collect(int vectorId, float similarity);
+
+  /**
+   * @return Is the current result set considered full
+   */
+  abstract boolean isFull();

Review Comment:
   I see what you are saying, but I think I still like the "trusting implementers" option better. The semantics I'd like for this method is something like "the minimum similarity for a vector to be competitive", so it would naturally be NEGATIVE_INFINITY as long as the queue is not full. If we don't trust implementers, then we need to update javadocs of `minSimilarity()` to add something like "it is only legal to call this method when isFull() returns true" which isn't nice, let's make `minSimilarity()` always correct? In terms of implementation, I imagine that `minSimilarity()` would need to do something like `queue.size() >= k() ? queue.topScore() : Float.NEGATIVE_INFINITY`? I see that we also use `minSimilarity()` on `GraphBuilderKnnResults` as a way to know the score of the top node, but I'm assuming we could address this by adding a new method on `GraphBuilderKnnResults`, since it's an implementation details of our HNSW impl?



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274933739


##########
lucene/core/src/java/org/apache/lucene/index/LeafReader.java:
##########
@@ -268,8 +272,68 @@ public abstract TopDocs searchNearestVectors(
    * @return the k nearest neighbor documents, along with their (searchStrategy-specific) scores.
    * @lucene.experimental
    */
+  public final TopDocs searchNearestVectors(
+      String field, byte[] target, int k, Bits acceptDocs, int visitedLimit) throws IOException {
+    return searchNearestVectors(field, target, new TopKnnResults(k, visitedLimit), acceptDocs);
+  }
+
+  /**
+   * Return the k nearest neighbor documents as determined by comparison of their vector values for
+   * this field, to the given vector, by the field's similarity function. The score of each document
+   * is derived from the vector similarity in a way that ensures scores are positive and that a
+   * larger score corresponds to a higher ranking.
+   *
+   * <p>The search is allowed to be approximate, meaning the results are not guaranteed to be the
+   * true k closest neighbors. For large values of k (for example when k is close to the total
+   * number of documents), the search may also retrieve fewer than k documents.
+   *
+   * <p>The returned {@link TopDocs} will contain a {@link ScoreDoc} for each nearest neighbor, in
+   * order of their similarity to the query vector (decreasing scores). The {@link TotalHits}
+   * contains the number of documents visited during the search. If the search stopped early because
+   * it hit {@code visitedLimit}, it is indicated through the relation {@code
+   * TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO}.
+   *
+   * <p>The behavior is undefined if the given field doesn't have KNN vectors enabled on its {@link
+   * FieldInfo}. The return value is never {@code null}.
+   *
+   * @param field the vector field to search
+   * @param target the vector-valued query
+   * @param knnResults collector and topK for gathering the vector results
+   * @param acceptDocs {@link Bits} that represents the allowed documents to match, or {@code null}
+   *     if they are all allowed to match.
+   * @return the k nearest neighbor documents, along with their (similarity-specific) scores.
+   */
+  public abstract TopDocs searchNearestVectors(

Review Comment:
   If we make the reader void, then this should be as well.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1275357936


##########
lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsReader.java:
##########
@@ -79,14 +80,13 @@ protected KnnVectorsReader() {}
    *
    * @param field the vector field to search
    * @param target the vector-valued query
-   * @param k the number of docs to return
+   * @param knnResults a KnnResults collector and relevant settings for gathering vector results
    * @param acceptDocs {@link Bits} that represents the allowed documents to match, or {@code null}
    *     if they are all allowed to match.
-   * @param visitedLimit the maximum number of nodes that the search is allowed to visit
    * @return the k nearest neighbor documents, along with their (similarity-specific) scores.
    */
   public abstract TopDocs search(
-      String field, float[] target, int k, Bits acceptDocs, int visitedLimit) throws IOException;
+      String field, float[] target, KnnResults knnResults, Bits acceptDocs) throws IOException;

Review Comment:
   Since this method accepts a collector, it should be void. Methods that do not accept a collector and instead create one itself continue to return `TopDocs` where appropriate.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274119017


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResults.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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 org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  /** KnnResults when exiting search early and returning empty top docs */
+  static class EmptyKnnResults extends KnnResults {
+    public EmptyKnnResults(int k, int visitedCount, int visitLimit) {
+      super(k, visitLimit);
+      this.visitedCount = visitedCount;
+    }
+
+    @Override
+    public void doClear() {}
+
+    @Override
+    public boolean collect(int vectorId, float similarity) {
+      throw new IllegalArgumentException();
+    }
+
+    @Override
+    public boolean isFull() {
+      return true;
+    }
+
+    @Override
+    public float minSimilarity() {
+      return 0;
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TotalHits th = new TotalHits(visitedCount, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO);
+      return new TopDocs(th, new ScoreDoc[0]);
+    }
+  }
+
+  static class OrdinalTranslatedKnnResults extends KnnResults {
+    private final KnnResults in;
+    private final IntToIntFunction vectorOrdinalToDocId;
+
+    OrdinalTranslatedKnnResults(KnnResults in, IntToIntFunction vectorOrdinalToDocId) {
+      super(in.k, in.visitLimit);
+      this.in = in;
+      this.vectorOrdinalToDocId = vectorOrdinalToDocId;
+    }
+
+    @Override
+    void doClear() {
+      in.clear();
+    }
+
+    @Override
+    boolean collect(int vectorId, float similarity) {
+      return in.collect(vectorOrdinalToDocId.apply(vectorId), similarity);
+    }
+
+    @Override
+    boolean isFull() {
+      return in.isFull();
+    }
+
+    @Override
+    float minSimilarity() {
+      return in.minSimilarity();
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TopDocs td = in.topDocs();
+      return new TopDocs(
+          new TotalHits(
+              visitedCount(),
+              incomplete()
+                  ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+                  : TotalHits.Relation.EQUAL_TO),
+          td.scoreDocs);
+    }
+  }
+
+  protected int visitedCount;
+  private final int visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, int visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  final void clear() {
+    this.visitedCount = 0;
+    doClear();
+  }
+
+  /** Clear the current results. */
+  abstract void doClear();
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  final boolean incomplete() {
+    return visitedCount >= visitLimit;
+  }
+
+  final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited count
+   */
+  final int visitedCount() {
+    return visitedCount;
+  }
+
+  final int visitLimit() {
+    return visitLimit;
+  }
+
+  public final int k() {
+    return k;
+  }
+
+  /**
+   * Collect the provided vectorId and include in the result set.
+   *
+   * @param vectorId the vector to collect
+   * @param similarity its calculated similarity
+   * @return true if the vector is collected
+   */
+  abstract boolean collect(int vectorId, float similarity);

Review Comment:
   updated



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1276260526


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/TopKnnCollector.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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;

Review Comment:
   OK. Then I'd still suggest moving this class to `oal.search` as it doesn't make sense for `LeafReader` to rely on HNSW-specific code, and also to make KnnCollector and the `LeafReader#searchNearestVector` method that takes a collector tagged with `@lucene .experimental` to convey that they are expert and may get changes in the future.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1276665351


##########
lucene/core/src/java/org/apache/lucene/search/TopKnnCollector.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.search;
+
+import org.apache.lucene.util.hnsw.NeighborQueue;
+
+/**
+ * TopKnnCollector is a specific KnnCollector. A minHeap is used to keep track of the currently
+ * collected vectors allowing for efficient updates as better vectors are collected.
+ *
+ * @lucene.experimental
+ */
+public final class TopKnnCollector extends AbstractKnnCollector {
+
+  private final NeighborQueue queue;
+
+  /**
+   * @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) {
+    super(k, visitLimit);
+    this.queue = new NeighborQueue(k, false);
+  }
+
+  @Override
+  public boolean collect(int docId, float similarity) {
+    return queue.insertWithOverflow(docId, similarity);
+  }
+
+  @Override
+  public float minCompetitiveSimilarity() {
+    return queue.size() >= k() ? queue.topScore() : Float.NEGATIVE_INFINITY;
+  }
+
+  @Override
+  public TopDocs topDocs() {
+    while (queue.size() > k()) {

Review Comment:
   Adding an assertion and removing this since we always insertWithOverflow.



##########
lucene/core/src/java/org/apache/lucene/search/TopKnnCollector.java:
##########
@@ -0,0 +1,75 @@
+/*
+ * 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.search;
+
+import org.apache.lucene.util.hnsw.NeighborQueue;
+
+/**
+ * TopKnnCollector is a specific KnnCollector. A minHeap is used to keep track of the currently
+ * collected vectors allowing for efficient updates as better vectors are collected.
+ *
+ * @lucene.experimental
+ */
+public final class TopKnnCollector extends AbstractKnnCollector {
+
+  private final NeighborQueue queue;
+
+  /**
+   * @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) {
+    super(k, visitLimit);
+    this.queue = new NeighborQueue(k, false);
+  }
+
+  @Override
+  public boolean collect(int docId, float similarity) {
+    return queue.insertWithOverflow(docId, similarity);
+  }
+
+  @Override
+  public float minCompetitiveSimilarity() {
+    return queue.size() >= k() ? queue.topScore() : Float.NEGATIVE_INFINITY;
+  }
+
+  @Override
+  public TopDocs topDocs() {
+    while (queue.size() > k()) {
+      queue.pop();
+    }
+    int i = 0;
+    ScoreDoc[] scoreDocs = new ScoreDoc[queue.size()];
+    while (i < scoreDocs.length) {

Review Comment:
   yep, changing that



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274861961


##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected int visitedCount;
+  private final int visitLimit;

Review Comment:
   Updated, they are both `long` value 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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1276167571


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/TopKnnCollector.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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;

Review Comment:
   @jpountz I think we can cross that bridge when we get to it. It might be that all additional options are statically set in the codec, or there is a good default that scales with `k`, or there is a new query type :/ I am just hesitant saying "this collector is true for forever" without knowing what a new codec other than HNSW might need.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1273966365


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResults.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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 org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  /** KnnResults when exiting search early and returning empty top docs */
+  static class EmptyKnnResults extends KnnResults {
+    public EmptyKnnResults(int k, int visitedCount, int visitLimit) {
+      super(k, visitLimit);
+      this.visitedCount = visitedCount;
+    }
+
+    @Override
+    public void doClear() {}
+
+    @Override
+    public boolean collect(int vectorId, float similarity) {
+      throw new IllegalArgumentException();
+    }
+
+    @Override
+    public boolean isFull() {
+      return true;
+    }
+
+    @Override
+    public float minSimilarity() {
+      return 0;
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TotalHits th = new TotalHits(visitedCount, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO);
+      return new TopDocs(th, new ScoreDoc[0]);
+    }
+  }
+
+  static class OrdinalTranslatedKnnResults extends KnnResults {
+    private final KnnResults in;
+    private final IntToIntFunction vectorOrdinalToDocId;
+
+    OrdinalTranslatedKnnResults(KnnResults in, IntToIntFunction vectorOrdinalToDocId) {
+      super(in.k, in.visitLimit);
+      this.in = in;
+      this.vectorOrdinalToDocId = vectorOrdinalToDocId;
+    }
+
+    @Override
+    void doClear() {
+      in.clear();
+    }
+
+    @Override
+    boolean collect(int vectorId, float similarity) {
+      return in.collect(vectorOrdinalToDocId.apply(vectorId), similarity);
+    }
+
+    @Override
+    boolean isFull() {
+      return in.isFull();
+    }
+
+    @Override
+    float minSimilarity() {
+      return in.minSimilarity();
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TopDocs td = in.topDocs();
+      return new TopDocs(
+          new TotalHits(
+              visitedCount(),
+              incomplete()
+                  ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+                  : TotalHits.Relation.EQUAL_TO),
+          td.scoreDocs);
+    }
+  }
+
+  protected int visitedCount;
+  private final int visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, int visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  final void clear() {
+    this.visitedCount = 0;
+    doClear();
+  }
+
+  /** Clear the current results. */
+  abstract void doClear();
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  final boolean incomplete() {
+    return visitedCount >= visitLimit;
+  }
+
+  final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited count
+   */
+  final int visitedCount() {
+    return visitedCount;
+  }
+
+  final int visitLimit() {
+    return visitLimit;
+  }
+
+  public final int k() {
+    return k;
+  }
+
+  /**
+   * Collect the provided vectorId and include in the result set.
+   *
+   * @param vectorId the vector to collect
+   * @param similarity its calculated similarity
+   * @return true if the vector is collected
+   */
+  abstract boolean collect(int vectorId, float similarity);

Review Comment:
   @jpountz it could be? But technically, the thing being passed in is still always the vectorId, we just have a results collector that does the translation before passing it into an inner collector.



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResults.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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 org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  /** KnnResults when exiting search early and returning empty top docs */
+  static class EmptyKnnResults extends KnnResults {
+    public EmptyKnnResults(int k, int visitedCount, int visitLimit) {
+      super(k, visitLimit);
+      this.visitedCount = visitedCount;
+    }
+
+    @Override
+    public void doClear() {}
+
+    @Override
+    public boolean collect(int vectorId, float similarity) {
+      throw new IllegalArgumentException();
+    }
+
+    @Override
+    public boolean isFull() {
+      return true;
+    }
+
+    @Override
+    public float minSimilarity() {
+      return 0;
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TotalHits th = new TotalHits(visitedCount, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO);
+      return new TopDocs(th, new ScoreDoc[0]);
+    }
+  }
+
+  static class OrdinalTranslatedKnnResults extends KnnResults {
+    private final KnnResults in;
+    private final IntToIntFunction vectorOrdinalToDocId;
+
+    OrdinalTranslatedKnnResults(KnnResults in, IntToIntFunction vectorOrdinalToDocId) {
+      super(in.k, in.visitLimit);
+      this.in = in;
+      this.vectorOrdinalToDocId = vectorOrdinalToDocId;
+    }
+
+    @Override
+    void doClear() {
+      in.clear();
+    }
+
+    @Override
+    boolean collect(int vectorId, float similarity) {
+      return in.collect(vectorOrdinalToDocId.apply(vectorId), similarity);
+    }
+
+    @Override
+    boolean isFull() {
+      return in.isFull();
+    }
+
+    @Override
+    float minSimilarity() {
+      return in.minSimilarity();
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TopDocs td = in.topDocs();
+      return new TopDocs(
+          new TotalHits(
+              visitedCount(),
+              incomplete()
+                  ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+                  : TotalHits.Relation.EQUAL_TO),
+          td.scoreDocs);
+    }
+  }
+
+  protected int visitedCount;
+  private final int visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, int visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  final void clear() {
+    this.visitedCount = 0;
+    doClear();
+  }
+
+  /** Clear the current results. */
+  abstract void doClear();

Review Comment:
   I will remove them.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274952636


##########
lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java:
##########
@@ -278,7 +276,7 @@ public TopDocs search(String field, float[] target, int k, Bits acceptDocs, int
   }
 
   @Override
-  public TopDocs search(String field, byte[] target, int k, Bits acceptDocs, int visitedLimit)
+  public TopDocs search(String field, byte[] target, KnnResults knnResults, Bits acceptDocs)

Review Comment:
   You are right, `ToParentJoinKnnResults` was not the best example. I'm mostly trying to think of whether bypassing the `KnnResults` object could yield issues and if we should collect entries from the result `NeighborQueue` into the `KnnResults` object.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] msokolov commented on pull request #12434: Add ParentJoin KNN support

Posted by "msokolov (via GitHub)" <gi...@apache.org>.
msokolov commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1656806007

   Hi @benwtrent I'm definitely curious to have a look at what you did here, but before diving in to the details (it's big!) I'd like to have a clearer sense of the problem you're solving. Is there a GH issue I'm missing that has more of a writeup? What confuses me is, I would have expected something like `ToParentBlockJoinQuery(parentBitSet, KnnVectorQuery())` to more or less work already. Clearly I'm missing something, but maybe you could explain it clearly for the slower ones among us :)


-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1273644497


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/ToParentJoinKnnResults.java:
##########
@@ -0,0 +1,303 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BitSet;
+
+/** parent joining knn results, vectorIds are deduplicated according to the parent bit set. */
+public class ToParentJoinKnnResults extends KnnResults {

Review Comment:
   Could this live in the lucene/join module instead of core? Likewise for `NodeIdCachingHeap`?



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1273669221


##########
lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsReader.java:
##########
@@ -117,6 +118,71 @@ public abstract TopDocs search(
    */
   public abstract TopDocs search(
       String field, byte[] target, int k, Bits acceptDocs, int visitedLimit) throws IOException;
+
+  /**
+   * Return the k nearest neighbor documents as determined by comparison of their vector values for
+   * this field, to the given vector, by the field's similarity function. The score of each document
+   * is derived from the vector similarity in a way that ensures scores are positive and that a
+   * larger score corresponds to a higher ranking.
+   *
+   * <p>The search is allowed to be approximate, meaning the results are not guaranteed to be the
+   * true k closest neighbors. For large values of k (for example when k is close to the total
+   * number of documents), the search may also retrieve fewer than k documents.
+   *
+   * <p>The returned {@link TopDocs} will contain a {@link ScoreDoc} for each nearest neighbor, in
+   * order of their similarity to the query vector (decreasing scores). The {@link TotalHits}
+   * contains the number of documents visited during the search. If the search stopped early because
+   * it hit {@code visitedLimit}, it is indicated through the relation {@code
+   * TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO}.
+   *
+   * <p>The behavior is undefined if the given field doesn't have KNN vectors enabled on its {@link
+   * FieldInfo}. The return value is never {@code null}.
+   *
+   * @param field the vector field to search
+   * @param target the vector-valued query
+   * @param knnResults a KnnResults collector and relevant settings for gathering vector results
+   * @param acceptDocs {@link Bits} that represents the allowed documents to match, or {@code null}
+   *     if they are all allowed to match.
+   * @return the k nearest neighbor documents, along with their (similarity-specific) scores.
+   */
+  public TopDocs search(String field, float[] target, KnnResults knnResults, Bits acceptDocs)
+      throws IOException {
+    throw new UnsupportedOperationException(
+        "vector reader doesn't provide KNN search with results provider");

Review Comment:
   My point of view on `LeafReader` and other related classes is that we should avoid breaking callers, but it's fine to break folks who extend these classes - as the latter is very expert usage of 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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1633057341

   @jpountz I took another shot at the KnnResults interface. I restricted the abstract and `@Override` methods to narrow the API. Additionally, I disconnected it from the queue, but it still has a queue object internally that sub-classes can utilize.


-- 
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-unsubscribe@lucene.apache.org

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


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


Re: [PR] Add ParentJoin KNN support [lucene]

Posted by "gauravj88 (via GitHub)" <gi...@apache.org>.
gauravj88 commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1844886604

   Hello,
   
   I hope this message finds you well. I am reaching out regarding the recent enhancement described in https://lucene.apache.org/core/9_8_0/changes/Changes.html#v9.8.0.new_features GitHub issue #12434, which introduces the KnnCollector to LeafReader and KnnVectorReader for custom collection of vector search results.
   
   I am currently using a KNN query in the following format:
   
   {!knn f=vector topK=5}[-0.08779877, ... , 0.009549321]
   I would like to leverage the new KnnCollector to obtain scores from child documents. Specifically, I would appreciate an example illustrating how to:
   
   Apply a filter query (fq) on the parent document.
   Retrieve both the parent document and child document with the child's score.
   Your assistance with this matter would be immensely helpful. Thank you in advance for your time and support.
   
   Best regards,
   Gaurav Jayswal


-- 
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-unsubscribe@lucene.apache.org

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


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


Re: [PR] Add ParentJoin KNN support [lucene]

Posted by "david-sitsky (via GitHub)" <gi...@apache.org>.
david-sitsky commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1836848114

   > @david-sitsky sorry for the confusion, it was renamed `DiversifyingChildren*KnnVectorQuery`
   
   Ah.. no worries, thanks.  We should update the changelog https://lucene.apache.org/core/9_8_0/changes/Changes.html#v9.8.0.new_features since it is still referring to the old classnames.


-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] msokolov commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "msokolov (via GitHub)" <gi...@apache.org>.
msokolov commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1279450672


##########
lucene/join/src/java/org/apache/lucene/search/join/ToParentJoinKnnCollector.java:
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.search.join;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.search.AbstractKnnCollector;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BitSet;
+
+/** parent joining knn collector, vector docIds are deduplicated according to the parent bit set. */
+class ToParentJoinKnnCollector extends AbstractKnnCollector {
+
+  private final BitSet parentBitSet;
+  private final NodeIdCachingHeap heap;
+
+  /**
+   * Create a new object for joining nearest child kNN documents with a parent bitset
+   *
+   * @param k The number of joined parent documents to collect
+   * @param visitLimit how many child vectors can be visited
+   * @param parentBitSet The leaf parent bitset
+   */
+  public ToParentJoinKnnCollector(int k, int visitLimit, BitSet parentBitSet) {
+    super(k, visitLimit);
+    this.parentBitSet = parentBitSet;
+    this.heap = new NodeIdCachingHeap(k);
+  }
+
+  /**
+   * If the heap is not full (size is less than the initialSize provided to the constructor), adds a
+   * new node-and-score element. If the heap is full, compares the score against the current top
+   * score, and replaces the top element if newScore is better than (greater than unless the heap is
+   * reversed), the current top score.
+   *
+   * <p>If docId's parent node has previously been collected and the provided nodeScore is less than
+   * the stored score it will not be collected.
+   *
+   * @param docId the neighbor docId
+   * @param nodeScore the score of the neighbor, relative to some other node
+   */
+  @Override
+  public boolean collect(int docId, float nodeScore) {
+    assert !parentBitSet.get(docId);
+    int nodeId = parentBitSet.nextSetBit(docId);
+    return heap.insertWithOverflow(nodeId, nodeScore);
+  }
+
+  @Override
+  public float minCompetitiveSimilarity() {
+    return heap.size >= k() ? heap.topScore() : Float.NEGATIVE_INFINITY;
+  }
+
+  @Override
+  public String toString() {
+    return "ToParentJoinKnnCollector[k=" + k() + ", size=" + heap.size() + "]";
+  }
+
+  @Override
+  public TopDocs topDocs() {
+    assert heap.size() <= k() : "Tried to collect more results than the maximum number allowed";
+    while (heap.size() > k()) {
+      heap.popToDrain();
+    }
+    ScoreDoc[] scoreDocs = new ScoreDoc[heap.size()];
+    for (int i = 1; i <= scoreDocs.length; i++) {
+      scoreDocs[scoreDocs.length - i] = new ScoreDoc(heap.topNode(), heap.topScore());
+      heap.popToDrain();
+    }
+
+    TotalHits.Relation relation =
+        earlyTerminated()
+            ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+            : TotalHits.Relation.EQUAL_TO;
+    return new TopDocs(new TotalHits(visitedCount(), relation), scoreDocs);
+  }
+
+  /**
+   * This is a minimum binary heap, inspired by {@link org.apache.lucene.util.LongHeap}. But instead
+   * of encoding and using `long` values. Node ids and scores are kept separate. Additionally, this
+   * prevents duplicate nodes from being added.
+   *
+   * <p>So, for every node added, we will update its score if the newly provided score is better.
+   * Every time we update a node's stored score, we ensure the heap's order.
+   */
+  private static class NodeIdCachingHeap {
+    private final int maxSize;
+    private int[] heapNodes;
+    private float[] heapScores;
+    private int size = 0;
+
+    // Used to keep track of nodeId -> positionInHeap. This way when new scores are added for a

Review Comment:
   Yeah sorry I was thinking about a different collector where we record *all* the child ids in order to later re-rank them. In that case we have an external map from nodeId -> grouping-tuple (We have to retain the tuples even after they are noncompetitive in case they become competitive later). Maybe the need for that map negates any benefit of this suggestion.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1658402079

   >  If it's a small number (say c children per parent), it may be better to use KNN search with K' = c * K. It would be interesting to compare these two approaches to see if we can provide some guidance or even some kind of api that chooses?
   
   @msokolov I have had similar thoughts. I think this type of optimization could be done even if there aren't any child filters. What about the case where every parent has a single child? It's definitely better to gather the vectors from the sparse index and then join back (.e.g post filter) on the parent docs.
   
   The calculation would be something like `K' = max(c) * K` and then join after gathering the results. This could be revisited after this initial work is completed.


-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1279318027


##########
lucene/core/src/test/org/apache/lucene/util/hnsw/TestNeighborQueue.java:
##########
@@ -114,6 +114,38 @@ public void testUnboundedQueue() {
     assertEquals(maxNode, nn.topNode());
   }
 
+  public void testCollectAndProvideResultsSameIds() {

Review Comment:
   @msokolov you are correct. This is an old test when I was experimenting with various ways to join against the parent docs while searching the graph.
   
   The current implementation doesn't need this guarantee. I will remove this test! 



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274829365


##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected int visitedCount;
+  private final int visitLimit;

Review Comment:
   Agreed. For reference, points have a similar challenge, see e.g. `PointValues#estimateDocCount` for instance which tries to convert a number of matching values to a number of matching docs.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274892128


##########
lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java:
##########
@@ -278,7 +276,7 @@ public TopDocs search(String field, float[] target, int k, Bits acceptDocs, int
   }
 
   @Override
-  public TopDocs search(String field, byte[] target, int k, Bits acceptDocs, int visitedLimit)
+  public TopDocs search(String field, byte[] target, KnnResults knnResults, Bits acceptDocs)

Review Comment:
   I'm not clear if this is a temporary thing and you plan on collecting vectors into the `KnnResults` object in a follow-up commit, or if this is the way things will be. I don't like that this is bypassing the `KnnResults` object, e.g. passing a `ToParentJoinKnnResults` would still return child doc IDs instead of parent doc IDs? I'm assuming it's fixable, but maybe this old codec makes it challenging?



##########
lucene/core/src/java/org/apache/lucene/index/LeafReader.java:
##########
@@ -268,8 +272,68 @@ public abstract TopDocs searchNearestVectors(
    * @return the k nearest neighbor documents, along with their (searchStrategy-specific) scores.
    * @lucene.experimental
    */
+  public final TopDocs searchNearestVectors(
+      String field, byte[] target, int k, Bits acceptDocs, int visitedLimit) throws IOException {
+    return searchNearestVectors(field, target, new TopKnnResults(k, visitedLimit), acceptDocs);
+  }
+
+  /**
+   * Return the k nearest neighbor documents as determined by comparison of their vector values for
+   * this field, to the given vector, by the field's similarity function. The score of each document
+   * is derived from the vector similarity in a way that ensures scores are positive and that a
+   * larger score corresponds to a higher ranking.
+   *
+   * <p>The search is allowed to be approximate, meaning the results are not guaranteed to be the
+   * true k closest neighbors. For large values of k (for example when k is close to the total
+   * number of documents), the search may also retrieve fewer than k documents.
+   *
+   * <p>The returned {@link TopDocs} will contain a {@link ScoreDoc} for each nearest neighbor, in
+   * order of their similarity to the query vector (decreasing scores). The {@link TotalHits}
+   * contains the number of documents visited during the search. If the search stopped early because
+   * it hit {@code visitedLimit}, it is indicated through the relation {@code
+   * TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO}.
+   *
+   * <p>The behavior is undefined if the given field doesn't have KNN vectors enabled on its {@link
+   * FieldInfo}. The return value is never {@code null}.
+   *
+   * @param field the vector field to search
+   * @param target the vector-valued query
+   * @param knnResults collector and topK for gathering the vector results
+   * @param acceptDocs {@link Bits} that represents the allowed documents to match, or {@code null}
+   *     if they are all allowed to match.
+   * @return the k nearest neighbor documents, along with their (similarity-specific) scores.
+   */
+  public abstract TopDocs searchNearestVectors(

Review Comment:
   should it return `void` as well and require callers to pull top docs on the `KnnResults` object?



##########
lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsReader.java:
##########
@@ -79,14 +80,13 @@ protected KnnVectorsReader() {}
    *
    * @param field the vector field to search
    * @param target the vector-valued query
-   * @param k the number of docs to return
+   * @param knnResults a KnnResults collector and relevant settings for gathering vector results
    * @param acceptDocs {@link Bits} that represents the allowed documents to match, or {@code null}
    *     if they are all allowed to match.
-   * @param visitedLimit the maximum number of nodes that the search is allowed to visit
    * @return the k nearest neighbor documents, along with their (similarity-specific) scores.
    */
   public abstract TopDocs search(
-      String field, float[] target, int k, Bits acceptDocs, int visitedLimit) throws IOException;
+      String field, float[] target, KnnResults knnResults, Bits acceptDocs) throws IOException;

Review Comment:
   Should these methods return `void`, and make callers responsible for pulling top docs from the `KnnResults` object?



##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered

Review Comment:
   I'm tempted to rename it to KnnCollector given this description. This would make sense to me as this is very similar to query collectors?



##########
lucene/core/src/java/org/apache/lucene/index/LeafReader.java:
##########
@@ -268,8 +272,68 @@ public abstract TopDocs searchNearestVectors(
    * @return the k nearest neighbor documents, along with their (searchStrategy-specific) scores.
    * @lucene.experimental
    */
+  public final TopDocs searchNearestVectors(
+      String field, byte[] target, int k, Bits acceptDocs, int visitedLimit) throws IOException {
+    return searchNearestVectors(field, target, new TopKnnResults(k, visitedLimit), acceptDocs);
+  }
+
+  /**
+   * Return the k nearest neighbor documents as determined by comparison of their vector values for
+   * this field, to the given vector, by the field's similarity function. The score of each document
+   * is derived from the vector similarity in a way that ensures scores are positive and that a
+   * larger score corresponds to a higher ranking.
+   *
+   * <p>The search is allowed to be approximate, meaning the results are not guaranteed to be the
+   * true k closest neighbors. For large values of k (for example when k is close to the total
+   * number of documents), the search may also retrieve fewer than k documents.
+   *
+   * <p>The returned {@link TopDocs} will contain a {@link ScoreDoc} for each nearest neighbor, in
+   * order of their similarity to the query vector (decreasing scores). The {@link TotalHits}
+   * contains the number of documents visited during the search. If the search stopped early because
+   * it hit {@code visitedLimit}, it is indicated through the relation {@code
+   * TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO}.
+   *
+   * <p>The behavior is undefined if the given field doesn't have KNN vectors enabled on its {@link
+   * FieldInfo}. The return value is never {@code null}.
+   *
+   * @param field the vector field to search
+   * @param target the vector-valued query
+   * @param knnResults collector and topK for gathering the vector results

Review Comment:
   maybe add something about the fact that these objects should not be reused across calls to `search`



##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected long visitedCount;

Review Comment:
   should it be private? sub-classes already have visitedCount() if they need to access the value?



##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected long visitedCount;
+  private final long visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, long visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  /**
+   * If search visits too many documents, the results collector will terminate early. Usually, this
+   * is due to some restricted filter on the document set.
+   *
+   * <p>When collection is earlyTerminated, the results are not a correct representation of k
+   * nearest neighbors.
+   *
+   * @return is the current result set marked as incomplete?
+   */
+  public final boolean earlyTerminated() {
+    return visitedCount >= visitLimit;
+  }
+
+  /**
+   * @param count increments the visited vector count, must be greater than 0.
+   */
+  public final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }

Review Comment:
   I wonder if we should keep all methods abstract in this base implementation. When we had default implementations like that in the past, it then made it hard to wrap these classes as you couldn't delegate `incVisitedCount` to the inner `KnnResults`. Maybe split it into `KnnResults` that defines the API and `AbstractKnnResults` that includes this base implementation?



##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected long visitedCount;
+  private final long visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, long visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  /**
+   * If search visits too many documents, the results collector will terminate early. Usually, this
+   * is due to some restricted filter on the document set.
+   *
+   * <p>When collection is earlyTerminated, the results are not a correct representation of k
+   * nearest neighbors.
+   *
+   * @return is the current result set marked as incomplete?
+   */
+  public final boolean earlyTerminated() {
+    return visitedCount >= visitLimit;
+  }
+
+  /**
+   * @param count increments the visited vector count, must be greater than 0.
+   */
+  public final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited vector count
+   */
+  public final long visitedCount() {
+    return visitedCount;
+  }
+
+  /**
+   * @return the visited vector limit
+   */
+  public final long visitLimit() {
+    return visitLimit;
+  }
+
+  /**
+   * @return the expected number of collected results
+   */
+  public final int k() {
+    return k;
+  }
+
+  /**
+   * Collect the provided docId and include in the result set.
+   *
+   * @param docId of the vector to collect
+   * @param similarity its calculated similarity
+   * @return true if the vector is collected

Review Comment:
   Maybe clarify what it means for a vector to be collected. My understanding is that the semantics we want are that if this returns `false` then `minCompetitiveSimilarity` is guaranteed not to have been updated. Thinking out loud: I wonder how much perf this gives, maybe this could return `void` and we'd pull `minCompetitiveSimilarity` all the time in the search logic.



##########
lucene/join/src/java/org/apache/lucene/search/join/ToParentJoinKnnResults.java:
##########
@@ -0,0 +1,296 @@
+/*
+ * 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.search.join;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.search.KnnResults;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BitSet;
+
+/** parent joining knn results, vectorIds are deduplicated according to the parent bit set. */
+public class ToParentJoinKnnResults extends KnnResults {

Review Comment:
   This class could be pkg-private? (or tagged with `@lucene.internal` if you'd like to be able to reuse it elsewhere)



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/TopKnnResults.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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 org.apache.lucene.search.KnnResults;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+
+/**
+ * TopKnnResults is a specific KnnResults. A minHeap is used to keep track of the currently
+ * collected vectors allowing for efficient updates as better vectors are collected.
+ */
+public class TopKnnResults extends KnnResults {
+
+  protected final NeighborQueue queue;
+
+  /**
+   * @param k the number of neighbors to collect
+   * @param visitLimit how many vector nodes the results are allowed to visit
+   */
+  public TopKnnResults(int k, int visitLimit) {
+    super(k, visitLimit);
+    this.queue = new NeighborQueue(k, false);
+  }
+
+  @Override
+  public boolean collect(int docId, float similarity) {
+    return queue.insertWithOverflow(docId, similarity);
+  }
+
+  @Override
+  public float minCompetitiveSimilarity() {
+    return queue.size() >= k() ? queue.topScore() : Float.NEGATIVE_INFINITY;
+  }
+
+  @Override
+  public TopDocs topDocs() {
+    while (queue.size() > k()) {
+      queue.pop();
+    }
+    int i = 0;
+    ScoreDoc[] scoreDocs = new ScoreDoc[queue.size()];
+    while (i < scoreDocs.length) {
+      int node = queue.topNode();
+      float score = queue.topScore();
+      queue.pop();
+      scoreDocs[scoreDocs.length - ++i] = new ScoreDoc(node, score);
+    }
+    TotalHits.Relation relation =
+        earlyTerminated()
+            ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+            : TotalHits.Relation.EQUAL_TO;
+    return new TopDocs(new TotalHits(visitedCount(), relation), scoreDocs);
+  }
+
+  @Override
+  public String toString() {
+    return "TopKnnResults[" + queue.size() + "]";

Review Comment:
   Without reading the code, I would have assumed the number to be the `k` rather than the size of the queue.
   
   ```suggestion
       return "TopKnnResults[k=" + k() + ", size=" + queue.size() + "]";
   ```



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1276640184


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/TopKnnCollector.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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;

Review Comment:
   done.



-- 
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-unsubscribe@lucene.apache.org

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


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


Re: [PR] Add ParentJoin KNN support [lucene]

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1750930644

   Thanks @benwtrent for this work! I finally had the chance to take a look.
   It's a lot and I see it's already merged, so I don't have any meaningful comment at the moment, but if I have time I'll dive into it in the future! (mostly when and if I resume the work on multi-valued, for which I am still waiting for fundings).
   The work here drastically changes the way also my Pull Request should look like right now.
   
   As a side note, do you happen to have any performance benchmark? I am quite curious as I always label nested docs approaches in Lucene to be 'slow', but having some facts (that potentially contradicts my statement) would be super cool!


-- 
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-unsubscribe@lucene.apache.org

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


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


Re: [PR] Add ParentJoin KNN support [lucene]

Posted by "alessandrobenedetti (via GitHub)" <gi...@apache.org>.
alessandrobenedetti commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1751034549

   > > The work here drastically changes the way also my Pull Request should look like right now.
   > 
   > Yes, I am sorry about that. But the good news is that the integration for multi-value vectors has some nicer APIs to take advantage of (e.g. KnnCollector) and it could possibly copy/paste the deduplicating nearest neighbor min-heap implementation.
   > 
   No worries at all! My work is still paused, looking for sponsors, so no harm! When I resume it as you said I may find benefits (and do improvements) to the new data structures added (I admint I got lost in the amount of KnnCollectors and similar classes added, but I'm super curious to explore each of them thoroughfully. 
   > > As a side note, do you happen to have any performance benchmark?
   > 
   > The following test was completed over 139004 documents with 768 float32 dimensions.
   > 
   > The statistics for the nested value distributions:
   > 
   > `1944` total unique documents `62.0` median number of nested values `71.50411522633745` mean number of nested values `309` max number of nested values `1` min number of nested values `2156.9469722481676` variance of nested values
   > 
   > ```
   > |                                        50th percentile latency |          knn-search-10-100 |   3.10031     |     ms |
   > |                                        90th percentile latency |          knn-search-10-100 |   3.5629      |     ms |
   > |                                        99th percentile latency |          knn-search-10-100 |   4.60912     |     ms |
   > |                                      99.9th percentile latency |          knn-search-10-100 |  14.322       |     ms |
   > |                                       100th percentile latency |          knn-search-10-100 |  72.6463      |     ms |
   > |                                        50th percentile latency |   knn-nested-search-10-100 |   6.2615      |     ms |
   > |                                        90th percentile latency |   knn-nested-search-10-100 |   6.95849     |     ms |
   > |                                        99th percentile latency |   knn-nested-search-10-100 |   7.8881      |     ms |
   > |                                      99.9th percentile latency |   knn-nested-search-10-100 |  12.0871      |     ms |
   > |                                       100th percentile latency |   knn-nested-search-10-100 |  57.9238      |     ms |
   > |                                        50th percentile latency |        knn-search-100-1000 |   7.30288     |     ms |
   > |                                        90th percentile latency |        knn-search-100-1000 |   8.18694     |     ms |
   > |                                        99th percentile latency |        knn-search-100-1000 |   9.23673     |     ms |
   > |                                      99.9th percentile latency |        knn-search-100-1000 |  18.7072      |     ms |
   > |                                       100th percentile latency |        knn-search-100-1000 |  23.8712      |     ms |
   > |                                        50th percentile latency | knn-search-nested-100-1000 |  26.6446      |     ms |
   > |                                        90th percentile latency | knn-search-nested-100-1000 |  38.2561      |     ms |
   > |                                        99th percentile latency | knn-search-nested-100-1000 |  44.3627      |     ms |
   > |                                      99.9th percentile latency | knn-search-nested-100-1000 |  51.1843      |     ms |
   > |                                       100th percentile latency | knn-search-nested-100-1000 |  52.0864      |     ms |
   > ```
   > 
   > GASP! Nested seems 2x to 4x slower!
   > 
   > But, keep in mind, we are eagerly joining! When I dug into the difference, I discovered that eagerly joining on this dataset meant we were visiting 3x to 5x more vectors. Consequently doing 3-5x more vector comparisons and deeper exploration of the graph. This lines up really nicely with the performance difference.
   > 
   > Since HNSW is `log(n)` the overall constant overhead of nested seems rather minor compared to the need to gather nearest vectors.
   > 
   > I am not sure these numbers are reflective of other nested/block-joining operations (like a term search).
   Interesting and thanks for the heads up, I hope to investigate this further as well in the future!
   


-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274552571


##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected int visitedCount;
+  private final int visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, int visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  public final boolean incomplete() {
+    return visitedCount >= visitLimit;
+  }
+
+  public final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited count
+   */
+  public final int visitedCount() {
+    return visitedCount;
+  }
+
+  public final int visitLimit() {
+    return visitLimit;
+  }
+
+  public final int k() {

Review Comment:
   I wonder if we can avoid adding it to the public API, the search logic doesn't need to know how many hits are being searched?



##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected int visitedCount;
+  private final int visitLimit;

Review Comment:
   Thinking out loud: maybe we should make these two quantities `long`s instead of `int`s so that we don't have to break it when introducing multi-value support.



##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected int visitedCount;
+  private final int visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, int visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  public final boolean incomplete() {
+    return visitedCount >= visitLimit;
+  }
+
+  public final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited count
+   */
+  public final int visitedCount() {
+    return visitedCount;
+  }
+
+  public final int visitLimit() {
+    return visitLimit;
+  }
+
+  public final int k() {
+    return k;
+  }
+
+  /**
+   * Collect the provided docId and include in the result set.
+   *
+   * @param docId of the vector to collect
+   * @param similarity its calculated similarity
+   * @return true if the vector is collected
+   */
+  public abstract boolean collect(int docId, float similarity);
+
+  /**
+   * @return Is the current result set considered full
+   */
+  public abstract boolean isFull();
+
+  /**
+   * @return the current minimum similarity in the collection
+   */
+  public abstract float minSimilarity();

Review Comment:
   Related to the discussion about `isFull()`, maybe a better name would be `minCompetitiveSimilarity` (akin to `Scorable#setMinCompetitiveScore`) to better convey that this is used for filtering rather than a getter of the similarity of the top entry of the queue.



##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected int visitedCount;
+  private final int visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, int visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  public final boolean incomplete() {

Review Comment:
   Thinking out loud, I wonder if `exhausted` would be a better name as NN results would be incomplete by nature due to the approximate nature of most algorithms.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1643999697

   @jpountz 
   Here are two JFRs. Both are force-merged to a single segment. One is nested (which each parent only have a single nested doc) the other is not-nested (so densely packed vectors and using the regular query).
   
   My tests show the current implementation (keeping track of hashes while building heap) is between 30-50% slower when doing nested queries.
   
   [jfrs.zip](https://github.com/apache/lucene/files/12110174/jfrs.zip)
   


-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274813661


##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected int visitedCount;
+  private final int visitLimit;

Review Comment:
   I can do that



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274931211


##########
lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsReader.java:
##########
@@ -79,14 +80,13 @@ protected KnnVectorsReader() {}
    *
    * @param field the vector field to search
    * @param target the vector-valued query
-   * @param k the number of docs to return
+   * @param knnResults a KnnResults collector and relevant settings for gathering vector results
    * @param acceptDocs {@link Bits} that represents the allowed documents to match, or {@code null}
    *     if they are all allowed to match.
-   * @param visitedLimit the maximum number of nodes that the search is allowed to visit
    * @return the k nearest neighbor documents, along with their (similarity-specific) scores.
    */
   public abstract TopDocs search(
-      String field, float[] target, int k, Bits acceptDocs, int visitedLimit) throws IOException;
+      String field, float[] target, KnnResults knnResults, Bits acceptDocs) throws IOException;

Review Comment:
   🤔 that seems ok to me.



##########
lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsReader.java:
##########
@@ -79,14 +80,13 @@ protected KnnVectorsReader() {}
    *
    * @param field the vector field to search
    * @param target the vector-valued query
-   * @param k the number of docs to return
+   * @param knnResults a KnnResults collector and relevant settings for gathering vector results
    * @param acceptDocs {@link Bits} that represents the allowed documents to match, or {@code null}
    *     if they are all allowed to match.
-   * @param visitedLimit the maximum number of nodes that the search is allowed to visit
    * @return the k nearest neighbor documents, along with their (similarity-specific) scores.
    */
   public abstract TopDocs search(
-      String field, float[] target, int k, Bits acceptDocs, int visitedLimit) throws IOException;
+      String field, float[] target, KnnResults knnResults, Bits acceptDocs) throws IOException;

Review Comment:
   🤔 that seems ok to me.



##########
lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsReader.java:
##########
@@ -79,14 +80,13 @@ protected KnnVectorsReader() {}
    *
    * @param field the vector field to search
    * @param target the vector-valued query
-   * @param k the number of docs to return
+   * @param knnResults a KnnResults collector and relevant settings for gathering vector results
    * @param acceptDocs {@link Bits} that represents the allowed documents to match, or {@code null}
    *     if they are all allowed to match.
-   * @param visitedLimit the maximum number of nodes that the search is allowed to visit
    * @return the k nearest neighbor documents, along with their (similarity-specific) scores.
    */
   public abstract TopDocs search(
-      String field, float[] target, int k, Bits acceptDocs, int visitedLimit) throws IOException;
+      String field, float[] target, KnnResults knnResults, Bits acceptDocs) throws IOException;

Review Comment:
   🤔 that seems ok to me. I don't have a strong opinion.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1275258761


##########
lucene/core/src/java/org/apache/lucene/search/KnnByteVectorQuery.java:
##########
@@ -77,8 +78,16 @@ public KnnByteVectorQuery(String field, byte[] target, int k, Query filter) {
   @Override
   protected TopDocs approximateSearch(LeafReaderContext context, Bits acceptDocs, int visitedLimit)
       throws IOException {
+    FieldInfo fi = context.reader().getFieldInfos().fieldInfo(field);
+    if (fi == null || fi.getVectorDimension() == 0) {
+      // The field does not exist or does not index vectors
+      return NO_RESULTS;
+    }
+    int k = Math.min(this.k, context.reader().getByteVectorValues(fi.name).size());

Review Comment:
   > that takes an int k, and call it here instead of the expert method that takes a KnnCollector?
   
   🤔 I suppose? This way this queries use the `final` method from the leafReader and don't have to worry about this.



##########
lucene/core/src/java/org/apache/lucene/search/KnnByteVectorQuery.java:
##########
@@ -77,8 +78,16 @@ public KnnByteVectorQuery(String field, byte[] target, int k, Query filter) {
   @Override
   protected TopDocs approximateSearch(LeafReaderContext context, Bits acceptDocs, int visitedLimit)
       throws IOException {
+    FieldInfo fi = context.reader().getFieldInfos().fieldInfo(field);
+    if (fi == null || fi.getVectorDimension() == 0) {
+      // The field does not exist or does not index vectors
+      return NO_RESULTS;
+    }
+    int k = Math.min(this.k, context.reader().getByteVectorValues(fi.name).size());

Review Comment:
   > that takes an int k, and call it here instead of the expert method that takes a KnnCollector?
   
   🤔 I suppose? This way this queries use the `final` method from the leafReader and don't have to worry about 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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1656874923

   Thanks for digging in @msokolov!
   
   > I'd like to have a clearer sense of the problem you're solving.
   
   This PR solves a similar, but different problem to: https://github.com/apache/lucene/issues/12313 
   
   Text embedding models have a token limit, so when processing larger text inputs, they need to be divided into passages. These passages share a common parent document. When users search for the "top-k" documents, they expect the initial parent document as the result, not just individual passages.
   
   A "multi-value" vector only partially solves this. The user will need to know the nearest passages across those documents to use in retrieval augmented generation. Multi-value fields cannot solve this as metadata needs to be associated with each vector to tie them to an originating passage, or better yet, some field containing the passage text itself. `join` seemed like a natural place to tackle this.
   
    - We get the top-k parent documents (e.g. the users larger chunk of text)
    - And can still get the nearest passages from that deduplicated set of parent documents
   
   Not to mention the nice flexibility we get (filtering on passage metadata, filtering on parent documents, hybrid scoring on the parent or child level, etc.)
   
   > What confuses me is, I would have expected something like `ToParentBlockJoinQuery(parentBitSet, KnnVectorQuery())` to more or less work already.
   
   The main issue is that it won't return the correct number of parent documents when the user requests the top-k parents based on their children vectors. If there are multiple children per parent, this approach may return fewer than k parent documents.


-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] msokolov commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "msokolov (via GitHub)" <gi...@apache.org>.
msokolov commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1278615147


##########
lucene/core/src/test/org/apache/lucene/util/hnsw/TestNeighborQueue.java:
##########
@@ -114,6 +114,38 @@ public void testUnboundedQueue() {
     assertEquals(maxNode, nn.topNode());
   }
 
+  public void testCollectAndProvideResultsSameIds() {

Review Comment:
   This is a strange test case? We don't usually expect to store the same node id multiple times do we?



##########
lucene/core/src/java/org/apache/lucene/search/KnnCollector.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.search;
+
+/**
+ * KnnCollector is a knn collector used for gathering kNN results and providing topDocs from the
+ * gathered neighbors
+ *
+ * @lucene.experimental
+ */
+public interface KnnCollector {

Review Comment:
   Aside from supporting this change, the new interface seems to allow for a lot of consolidation, +1



##########
lucene/core/src/java/org/apache/lucene/util/hnsw/RandomAccessVectorValues.java:
##########
@@ -46,4 +46,14 @@ public interface RandomAccessVectorValues<T> {
    * {@link RandomAccessVectorValues#vectorValue}.
    */
   RandomAccessVectorValues<T> copy() throws IOException;
+
+  /**
+   * Translates vector ordinal to the correct document ID. By default, this is an identity function.
+   *
+   * @param ord the vector ordinal
+   * @return the document Id for that vector ordinal
+   */
+  default int ordToDoc(int ord) {

Review Comment:
   I think there might be some useless overrides after this?



##########
lucene/join/src/java/org/apache/lucene/search/join/ToParentJoinKnnCollector.java:
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.search.join;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.search.AbstractKnnCollector;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BitSet;
+
+/** parent joining knn collector, vector docIds are deduplicated according to the parent bit set. */
+class ToParentJoinKnnCollector extends AbstractKnnCollector {
+
+  private final BitSet parentBitSet;
+  private final NodeIdCachingHeap heap;
+
+  /**
+   * Create a new object for joining nearest child kNN documents with a parent bitset
+   *
+   * @param k The number of joined parent documents to collect
+   * @param visitLimit how many child vectors can be visited
+   * @param parentBitSet The leaf parent bitset
+   */
+  public ToParentJoinKnnCollector(int k, int visitLimit, BitSet parentBitSet) {
+    super(k, visitLimit);
+    this.parentBitSet = parentBitSet;
+    this.heap = new NodeIdCachingHeap(k);
+  }
+
+  /**
+   * If the heap is not full (size is less than the initialSize provided to the constructor), adds a
+   * new node-and-score element. If the heap is full, compares the score against the current top
+   * score, and replaces the top element if newScore is better than (greater than unless the heap is
+   * reversed), the current top score.
+   *
+   * <p>If docId's parent node has previously been collected and the provided nodeScore is less than
+   * the stored score it will not be collected.
+   *
+   * @param docId the neighbor docId
+   * @param nodeScore the score of the neighbor, relative to some other node
+   */
+  @Override
+  public boolean collect(int docId, float nodeScore) {
+    assert !parentBitSet.get(docId);
+    int nodeId = parentBitSet.nextSetBit(docId);
+    return heap.insertWithOverflow(nodeId, nodeScore);
+  }
+
+  @Override
+  public float minCompetitiveSimilarity() {
+    return heap.size >= k() ? heap.topScore() : Float.NEGATIVE_INFINITY;
+  }
+
+  @Override
+  public String toString() {
+    return "ToParentJoinKnnCollector[k=" + k() + ", size=" + heap.size() + "]";
+  }
+
+  @Override
+  public TopDocs topDocs() {
+    assert heap.size() <= k() : "Tried to collect more results than the maximum number allowed";
+    while (heap.size() > k()) {
+      heap.popToDrain();
+    }
+    ScoreDoc[] scoreDocs = new ScoreDoc[heap.size()];
+    for (int i = 1; i <= scoreDocs.length; i++) {
+      scoreDocs[scoreDocs.length - i] = new ScoreDoc(heap.topNode(), heap.topScore());
+      heap.popToDrain();
+    }
+
+    TotalHits.Relation relation =
+        earlyTerminated()
+            ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+            : TotalHits.Relation.EQUAL_TO;
+    return new TopDocs(new TotalHits(visitedCount(), relation), scoreDocs);
+  }
+
+  /**
+   * This is a minimum binary heap, inspired by {@link org.apache.lucene.util.LongHeap}. But instead
+   * of encoding and using `long` values. Node ids and scores are kept separate. Additionally, this
+   * prevents duplicate nodes from being added.
+   *
+   * <p>So, for every node added, we will update its score if the newly provided score is better.
+   * Every time we update a node's stored score, we ensure the heap's order.
+   */
+  private static class NodeIdCachingHeap {
+    private final int maxSize;
+    private int[] heapNodes;
+    private float[] heapScores;
+    private int size = 0;
+
+    // Used to keep track of nodeId -> positionInHeap. This way when new scores are added for a

Review Comment:
   did you consider having a heap of (node, score, position) tuples? I don't think there's any theoretical complexity difference (it's all O(1) insert into heap vs. updating a record) but it might be a little neater, requiring fewer parallel data structures.



##########
lucene/core/src/java/org/apache/lucene/codecs/lucene95/OffHeapFloatVectorValues.java:
##########
@@ -62,8 +62,6 @@ public float[] vectorValue(int targetOrd) throws IOException {
     return value;
   }
 
-  public abstract int ordToDoc(int ord);

Review Comment:
   can we also delete the overrides in the Dense* subclasses?



##########
lucene/core/src/java/org/apache/lucene/search/KnnCollector.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.search;
+
+/**
+ * KnnCollector is a knn collector used for gathering kNN results and providing topDocs from the
+ * gathered neighbors
+ *
+ * @lucene.experimental
+ */
+public interface KnnCollector {
+
+  /**
+   * If search visits too many documents, the results collector will terminate early. Usually, this
+   * is due to some restricted filter on the document set.
+   *
+   * <p>When collection is earlyTerminated, the results are not a correct representation of k
+   * nearest neighbors.
+   *
+   * @return is the current result set marked as incomplete?
+   */
+  boolean earlyTerminated();
+
+  /**
+   * @param count increments the visited vector count, must be greater than 0.
+   */
+  void incVisitedCount(int count);
+
+  /**
+   * @return the current visited vector count
+   */
+  long visitedCount();
+
+  /**
+   * @return the visited vector limit
+   */
+  long visitLimit();
+
+  /**
+   * @return the expected number of collected results
+   */
+  int k();
+
+  /**
+   * Collect the provided docId and include in the result set.

Review Comment:
   I think we are collecting vector ordinals, not docids. The javadoc should be clear about that so future implementors don't get confused!



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1279430631


##########
lucene/core/src/java/org/apache/lucene/search/KnnCollector.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.search;
+
+/**
+ * KnnCollector is a knn collector used for gathering kNN results and providing topDocs from the
+ * gathered neighbors
+ *
+ * @lucene.experimental
+ */
+public interface KnnCollector {
+
+  /**
+   * If search visits too many documents, the results collector will terminate early. Usually, this
+   * is due to some restricted filter on the document set.
+   *
+   * <p>When collection is earlyTerminated, the results are not a correct representation of k
+   * nearest neighbors.
+   *
+   * @return is the current result set marked as incomplete?
+   */
+  boolean earlyTerminated();
+
+  /**
+   * @param count increments the visited vector count, must be greater than 0.
+   */
+  void incVisitedCount(int count);
+
+  /**
+   * @return the current visited vector count
+   */
+  long visitedCount();
+
+  /**
+   * @return the visited vector limit
+   */
+  long visitLimit();
+
+  /**
+   * @return the expected number of collected results
+   */
+  int k();
+
+  /**
+   * Collect the provided docId and include in the result set.

Review Comment:
   @msokolov see: 
   
    - https://github.com/apache/lucene/pull/12434#discussion_r1260329978
    - https://github.com/apache/lucene/pull/12434/files#diff-ba3b6b03fc9c4dde56640064013726c8d5334076a6f8622c5f9eaee048b4ad47R272
   
   So, all user provided collectors will end up getting wrapped with the "OrdinalTranslatedKnnCollector" and the collection will only ever see docId. 
   
   Do you think this is a bad idea? If so, we need to provide the ordinal translation to the collector at some point.
   
   The requirements are:
    - Vector search over sparse indices requires the translation when building topDocs
    - Parent Join requires the translation eagerly because the parentBitSet assumes docIds, not sparse ordinals.
   
   I couldn't think of a clean way. Maybe having a "setter" method that can set the ordinal translation for the collector? Or we continue to do it eagerly?



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1666945234

   @msokolov let me know if there are further changes required.


-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274934075


##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,105 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered

Review Comment:
   I am fine with the unified naming.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1275240534


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/TopKnnCollector.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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;

Review Comment:
   @jpountz I am not sure, other nearest neighbor algos (like ivfpq) require more parameters that we currently don't provide. If we were to do something other than HNSW, I think a different collector would be required providing separate options.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274837238


##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected int visitedCount;
+  private final int visitLimit;

Review Comment:
   @jpountz , Ah, so multi-value would have some inverted `estimateVisitedVectors` given some filtered set, and that could be greater than `int`. Indeed, a `long` here would be useful in that case.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1275153663


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/TopKnnCollector.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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 org.apache.lucene.search.AbstractKnnCollector;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+
+/**
+ * TopKnnCollector is a specific KnnResults. A minHeap is used to keep track of the currently
+ * collected vectors allowing for efficient updates as better vectors are collected.
+ */
+public final class TopKnnCollector extends AbstractKnnCollector {
+
+  private final NeighborQueue queue;
+
+  /**
+   * @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) {
+    super(k, visitLimit);
+    this.queue = new NeighborQueue(k, false);
+  }
+
+  @Override
+  public boolean collect(int docId, float similarity) {
+    return queue.insertWithOverflow(docId, similarity);
+  }
+
+  @Override
+  public float minCompetitiveSimilarity() {
+    return queue.size() >= k() ? queue.topScore() : Float.NEGATIVE_INFINITY;
+  }
+
+  @Override
+  public TopDocs topDocs() {
+    while (queue.size() > k()) {
+      queue.pop();
+    }
+    int i = 0;
+    ScoreDoc[] scoreDocs = new ScoreDoc[queue.size()];
+    while (i < scoreDocs.length) {
+      int node = queue.topNode();
+      float score = queue.topScore();
+      queue.pop();
+      scoreDocs[scoreDocs.length - ++i] = new ScoreDoc(node, score);
+    }
+    TotalHits.Relation relation =
+        earlyTerminated()
+            ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+            : TotalHits.Relation.EQUAL_TO;
+    return new TopDocs(new TotalHits(visitedCount(), relation), scoreDocs);
+  }
+
+  @Override
+  public String toString() {
+    return "TopKnnResults[" + queue.size() + "]";

Review Comment:
   ```suggestion
       return "TopKnnResults[k=" + k() + ", size=" + queue.size() + "]";
   ```



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1272735873


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResultsProvider.java:
##########
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+/** knn results provider */
+public interface KnnResultsProvider {

Review Comment:
   I removed the provider idea and have only KnnResults



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274821505


##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected int visitedCount;
+  private final int visitLimit;

Review Comment:
   @jpountz, visitLimit is always "<maxDoc" it is only used to allow early termination based on some restrictive filter. 
   
   I honestly don't know what it looks like for multi-value and handling this. Right now `visitLimit` and `visitCount` relate to "this many vector calculations". In `multi-value` this doesn't easy relate to some `cost()` related to a filter.
   
   This contrasts with join as a childFilter is still applicable, and thus visitLimit/visitCount tracking being "this many vector calculations makes sense.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1272680226


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResults.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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 org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  /** KnnResults when exiting search early and returning empty top docs */
+  static class EmptyKnnResults extends KnnResults {
+    public EmptyKnnResults(int visitedCount) {
+      this.visitedCount = visitedCount;
+    }
+
+    @Override
+    public void doClear() {}
+
+    @Override
+    public void collect(int vectorId, float similarity) {
+      throw new IllegalArgumentException();
+    }
+
+    @Override
+    public boolean collectWithOverflow(int vectorId, float similarity) {
+      return false;
+    }
+
+    @Override
+    public boolean isFull() {
+      return true;
+    }
+
+    @Override
+    public float minSimilarity() {
+      return 0;
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TotalHits th = new TotalHits(visitedCount, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO);
+      return new TopDocs(th, new ScoreDoc[0]);
+    }
+  }
+
+  protected int visitedCount;
+  private boolean incomplete;
+
+  final void clear() {
+    this.visitedCount = 0;
+    this.incomplete = false;
+    doClear();
+  }
+
+  /** Clear the current results. */
+  abstract void doClear();
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  final boolean incomplete() {
+    return incomplete;
+  }
+
+  /** Mark the current result set as incomplete */
+  final void markIncomplete() {
+    this.incomplete = true;
+  }
+
+  /**
+   * @param count set the current visited count to the provided value
+   */
+  final void setVisitedCount(int count) {
+    this.visitedCount = count;
+  }
+
+  /**
+   * @return the current visited count
+   */
+  final int visitedCount() {
+    return visitedCount;
+  }
+
+  /**
+   * Collect the provided vectorId and include in the result set.
+   *
+   * @param vectorId the vector to collect
+   * @param similarity its calculated similarity
+   */
+  abstract void collect(int vectorId, float similarity);
+
+  /**
+   * @param vectorId the vector to collect
+   * @param similarity its calculated similarity
+   * @return true if the vector is collected
+   */
+  abstract boolean collectWithOverflow(int vectorId, float similarity);

Review Comment:
   Latest commit makes this change.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1273965669


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResults.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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 org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  /** KnnResults when exiting search early and returning empty top docs */
+  static class EmptyKnnResults extends KnnResults {
+    public EmptyKnnResults(int k, int visitedCount, int visitLimit) {
+      super(k, visitLimit);
+      this.visitedCount = visitedCount;
+    }
+
+    @Override
+    public void doClear() {}
+
+    @Override
+    public boolean collect(int vectorId, float similarity) {
+      throw new IllegalArgumentException();
+    }
+
+    @Override
+    public boolean isFull() {
+      return true;
+    }
+
+    @Override
+    public float minSimilarity() {
+      return 0;
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TotalHits th = new TotalHits(visitedCount, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO);
+      return new TopDocs(th, new ScoreDoc[0]);
+    }
+  }
+
+  static class OrdinalTranslatedKnnResults extends KnnResults {
+    private final KnnResults in;
+    private final IntToIntFunction vectorOrdinalToDocId;
+
+    OrdinalTranslatedKnnResults(KnnResults in, IntToIntFunction vectorOrdinalToDocId) {
+      super(in.k, in.visitLimit);
+      this.in = in;
+      this.vectorOrdinalToDocId = vectorOrdinalToDocId;
+    }
+
+    @Override
+    void doClear() {
+      in.clear();
+    }
+
+    @Override
+    boolean collect(int vectorId, float similarity) {
+      return in.collect(vectorOrdinalToDocId.apply(vectorId), similarity);
+    }
+
+    @Override
+    boolean isFull() {
+      return in.isFull();
+    }
+
+    @Override
+    float minSimilarity() {
+      return in.minSimilarity();
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TopDocs td = in.topDocs();
+      return new TopDocs(
+          new TotalHits(
+              visitedCount(),
+              incomplete()
+                  ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+                  : TotalHits.Relation.EQUAL_TO),
+          td.scoreDocs);
+    }
+  }
+
+  protected int visitedCount;
+  private final int visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, int visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  final void clear() {
+    this.visitedCount = 0;
+    doClear();
+  }
+
+  /** Clear the current results. */
+  abstract void doClear();
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  final boolean incomplete() {
+    return visitedCount >= visitLimit;
+  }
+
+  final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited count
+   */
+  final int visitedCount() {
+    return visitedCount;
+  }
+
+  final int visitLimit() {
+    return visitLimit;
+  }
+
+  public final int k() {
+    return k;
+  }
+
+  /**
+   * Collect the provided vectorId and include in the result set.
+   *
+   * @param vectorId the vector to collect
+   * @param similarity its calculated similarity
+   * @return true if the vector is collected
+   */
+  abstract boolean collect(int vectorId, float similarity);
+
+  /**
+   * @return Is the current result set considered full
+   */
+  abstract boolean isFull();

Review Comment:
   Keeping `isFull()` seems best to me of the available options.
   
   ### Trusting implementers 
   We can either trust that `KnnResults` implementers do the correct thing (which is return `NEGATIVE_INFINITY` when they are not yet full), or we need some method indicating if the result set is full.
   
   The difficult part is that if somebody creates a `KnnResults` object, and doesn't implement `minSimilarity()` correctly, it will break kNN search.
   
   ### Restricting implementation
   
   If we restrict `minSimilarity()` to `final` within `KnnResults`, the parent object needs something to indicate that it is `full` via some abstract method && some abstract method for getting the current minimum similarity (regardless of queue satisfaction). 
   
   So, even here, we would need some `isFull` method. It just might not be public (maybe it could be `protected`).



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] msokolov commented on pull request #12434: Add ParentJoin KNN support

Posted by "msokolov (via GitHub)" <gi...@apache.org>.
msokolov commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1656953197

   > The main issue is that it won't return the correct number of parent documents when the user requests the top-k parents based on their children vectors. If there are multiple children per parent, this approach may return fewer than k parent documents.
   
   Thanks I see now. So this is kind of similar in spirit to the existing problem where we want the top K documents (by vector distance) satisfying some constraints and we have to choose whether to find some higher number of nearest docs (K') in the hopes that at least K of them will satisfy the constraints (post-filtering), or whether to apply the filters while searching, guaranteeing top K. I just want to note that both approaches have merit; it's a tradeoff depending on how restrictive the filters are, but for not-very-restrictive filters, post-filtering can outperform. In this case I guess there is a similar tradeoff relating to how many child documents there typically are. If it's a small number (say c children per parent), it may be better to use KNN search with K' = c * K.  It would be interesting to compare these two approaches to see if we can provide some guidance or even some kind of api that chooses?


-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1279321413


##########
lucene/join/src/java/org/apache/lucene/search/join/ToParentJoinKnnCollector.java:
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.search.join;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.search.AbstractKnnCollector;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BitSet;
+
+/** parent joining knn collector, vector docIds are deduplicated according to the parent bit set. */
+class ToParentJoinKnnCollector extends AbstractKnnCollector {
+
+  private final BitSet parentBitSet;
+  private final NodeIdCachingHeap heap;
+
+  /**
+   * Create a new object for joining nearest child kNN documents with a parent bitset
+   *
+   * @param k The number of joined parent documents to collect
+   * @param visitLimit how many child vectors can be visited
+   * @param parentBitSet The leaf parent bitset
+   */
+  public ToParentJoinKnnCollector(int k, int visitLimit, BitSet parentBitSet) {
+    super(k, visitLimit);
+    this.parentBitSet = parentBitSet;
+    this.heap = new NodeIdCachingHeap(k);
+  }
+
+  /**
+   * If the heap is not full (size is less than the initialSize provided to the constructor), adds a
+   * new node-and-score element. If the heap is full, compares the score against the current top
+   * score, and replaces the top element if newScore is better than (greater than unless the heap is
+   * reversed), the current top score.
+   *
+   * <p>If docId's parent node has previously been collected and the provided nodeScore is less than
+   * the stored score it will not be collected.
+   *
+   * @param docId the neighbor docId
+   * @param nodeScore the score of the neighbor, relative to some other node
+   */
+  @Override
+  public boolean collect(int docId, float nodeScore) {
+    assert !parentBitSet.get(docId);
+    int nodeId = parentBitSet.nextSetBit(docId);
+    return heap.insertWithOverflow(nodeId, nodeScore);
+  }
+
+  @Override
+  public float minCompetitiveSimilarity() {
+    return heap.size >= k() ? heap.topScore() : Float.NEGATIVE_INFINITY;
+  }
+
+  @Override
+  public String toString() {
+    return "ToParentJoinKnnCollector[k=" + k() + ", size=" + heap.size() + "]";
+  }
+
+  @Override
+  public TopDocs topDocs() {
+    assert heap.size() <= k() : "Tried to collect more results than the maximum number allowed";
+    while (heap.size() > k()) {
+      heap.popToDrain();
+    }
+    ScoreDoc[] scoreDocs = new ScoreDoc[heap.size()];
+    for (int i = 1; i <= scoreDocs.length; i++) {
+      scoreDocs[scoreDocs.length - i] = new ScoreDoc(heap.topNode(), heap.topScore());
+      heap.popToDrain();
+    }
+
+    TotalHits.Relation relation =
+        earlyTerminated()
+            ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+            : TotalHits.Relation.EQUAL_TO;
+    return new TopDocs(new TotalHits(visitedCount(), relation), scoreDocs);
+  }
+
+  /**
+   * This is a minimum binary heap, inspired by {@link org.apache.lucene.util.LongHeap}. But instead
+   * of encoding and using `long` values. Node ids and scores are kept separate. Additionally, this
+   * prevents duplicate nodes from being added.
+   *
+   * <p>So, for every node added, we will update its score if the newly provided score is better.
+   * Every time we update a node's stored score, we ensure the heap's order.
+   */
+  private static class NodeIdCachingHeap {
+    private final int maxSize;
+    private int[] heapNodes;
+    private float[] heapScores;
+    private int size = 0;
+
+    // Used to keep track of nodeId -> positionInHeap. This way when new scores are added for a

Review Comment:
   > did you consider having a heap of (node, score, position) tuples?
   
   I did not. I can look into it. The reason for primitive value heaps was to prevent creating & destroying boxed objects but we run into that problem already because the hashmap requires boxed integers :).



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1632431376

   @alessandrobenedetti I took some of your ideas on deduplicating vector IDs based on some other id for this PR. If this work continues, I think some of it can transfer to the native multi-vector support in 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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1269789239


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/ToParentJoinKnnResults.java:
##########
@@ -0,0 +1,362 @@
+/*
+ * 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.HashMap;
+import java.util.Map;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BitSet;
+
+/** parent joining knn results, vectorIds are deduplicated according to the parent bit set. */
+public class ToParentJoinKnnResults extends KnnResults {
+
+  /** provider class for creating a new {@link ToParentJoinKnnResults} */
+  public static class Provider implements KnnResultsProvider {
+
+    private final int k;
+    private final BitSet parentBitSet;
+
+    public Provider(int k, BitSet parentBitSet) {
+      this.k = k;
+      this.parentBitSet = parentBitSet;
+    }
+
+    @Override
+    public int k() {
+      return k;
+    }
+
+    @Override
+    public KnnResults getKnnResults(IntToIntFunction vectorToOrd) {
+      return new ToParentJoinKnnResults(k, parentBitSet, vectorToOrd);
+    }
+  }
+
+  private final BitSet parentBitSet;
+  private final int k;
+  private final IntToIntFunction vectorToOrd;
+  private final NodeIdCachingHeap heap;
+
+  public ToParentJoinKnnResults(int k, BitSet parentBitSet, IntToIntFunction vectorToOrd) {
+    this.parentBitSet = parentBitSet;
+    this.k = k;
+    this.vectorToOrd = vectorToOrd;
+    this.heap = new NodeIdCachingHeap(k);
+  }
+
+  /**
+   * Adds a new graph arc, extending the storage as needed.
+   *
+   * <p>If the provided childNodeId's parent has been previously collected and the nodeScore is less
+   * than the previously stored score, this node will not be added to the collection.
+   *
+   * @param childNodeId the neighbor node id
+   * @param nodeScore the score of the neighbor, relative to some other node
+   */
+  @Override
+  public void collect(int childNodeId, float nodeScore) {
+    childNodeId = vectorToOrd.apply(childNodeId);
+    assert !parentBitSet.get(childNodeId);
+    int nodeId = parentBitSet.nextSetBit(childNodeId);
+    heap.push(nodeId, nodeScore);
+  }
+
+  /**
+   * If the heap is not full (size is less than the initialSize provided to the constructor), adds a
+   * new node-and-score element. If the heap is full, compares the score against the current top
+   * score, and replaces the top element if newScore is better than (greater than unless the heap is
+   * reversed), the current top score.
+   *
+   * <p>If childNodeId's parent node has previously been collected and the provided nodeScore is
+   * less than the stored score it will not be collected.
+   *
+   * @param childNodeId the neighbor node id
+   * @param nodeScore the score of the neighbor, relative to some other node
+   */
+  @Override
+  public boolean collectWithOverflow(int childNodeId, float nodeScore) {
+    // Parent and child nodes should be disjoint sets parent bit set should never have a child node
+    // ID present
+    childNodeId = vectorToOrd.apply(childNodeId);
+    assert !parentBitSet.get(childNodeId);
+    int nodeId = parentBitSet.nextSetBit(childNodeId);
+    return heap.insertWithOverflow(nodeId, nodeScore);
+  }
+
+  @Override
+  public boolean isFull() {
+    return heap.size >= k;
+  }
+
+  @Override
+  public float minSimilarity() {
+    return heap.topScore();
+  }
+
+  @Override
+  public void doClear() {
+    heap.clear();
+  }
+
+  @Override
+  public String toString() {
+    return "ToParentJoinKnnResults[" + heap.size + "]";
+  }
+
+  @Override
+  public TopDocs topDocs() {
+    while (heap.size() > k) {
+      heap.popToDrain();
+    }
+    int i = 0;
+    ScoreDoc[] scoreDocs = new ScoreDoc[heap.size()];
+    while (i < scoreDocs.length) {
+      int node = heap.topNode();
+      float score = heap.topScore();
+      heap.popToDrain();
+      scoreDocs[scoreDocs.length - ++i] = new ScoreDoc(node, score);
+    }
+
+    TotalHits.Relation relation =
+        incomplete() ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO : TotalHits.Relation.EQUAL_TO;
+    return new TopDocs(new TotalHits(visitedCount(), relation), scoreDocs);

Review Comment:
   `visitedCount()` is always weird to me anyways as its the number of vectors we compared the query vector with, not necessarily the number of valid vectors within the index.
   
   Knowing the number of vector comparisons is useful, but it definitely doesn't mean the same thing as other "TotalHits" calculations :/. I guess vector search has hijacked this class to mean something else. 



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1273656300


##########
lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsReader.java:
##########
@@ -117,6 +118,71 @@ public abstract TopDocs search(
    */
   public abstract TopDocs search(
       String field, byte[] target, int k, Bits acceptDocs, int visitedLimit) throws IOException;
+
+  /**
+   * Return the k nearest neighbor documents as determined by comparison of their vector values for
+   * this field, to the given vector, by the field's similarity function. The score of each document
+   * is derived from the vector similarity in a way that ensures scores are positive and that a
+   * larger score corresponds to a higher ranking.
+   *
+   * <p>The search is allowed to be approximate, meaning the results are not guaranteed to be the
+   * true k closest neighbors. For large values of k (for example when k is close to the total
+   * number of documents), the search may also retrieve fewer than k documents.
+   *
+   * <p>The returned {@link TopDocs} will contain a {@link ScoreDoc} for each nearest neighbor, in
+   * order of their similarity to the query vector (decreasing scores). The {@link TotalHits}
+   * contains the number of documents visited during the search. If the search stopped early because
+   * it hit {@code visitedLimit}, it is indicated through the relation {@code
+   * TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO}.
+   *
+   * <p>The behavior is undefined if the given field doesn't have KNN vectors enabled on its {@link
+   * FieldInfo}. The return value is never {@code null}.
+   *
+   * @param field the vector field to search
+   * @param target the vector-valued query
+   * @param knnResults a KnnResults collector and relevant settings for gathering vector results
+   * @param acceptDocs {@link Bits} that represents the allowed documents to match, or {@code null}
+   *     if they are all allowed to match.
+   * @return the k nearest neighbor documents, along with their (similarity-specific) scores.
+   */
+  public TopDocs search(String field, float[] target, KnnResults knnResults, Bits acceptDocs)
+      throws IOException {
+    throw new UnsupportedOperationException(
+        "vector reader doesn't provide KNN search with results provider");

Review Comment:
   @jpountz my concern is backporting. Wouldn't this break backwards compatibility? LeafReader && KnnVectorsReader are both exposed in the module definition and thus can be used by consumers via sub-classes.
   
   Or is backwards compatibility only a big concern on "normal" usage (e.g. not sub-classing)?



-- 
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-unsubscribe@lucene.apache.org

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


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


Re: [PR] Add ParentJoin KNN support [lucene]

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1750978708

   > The work here drastically changes the way also my Pull Request should look like right now.
   
   Yes, I am sorry about that. But the good news is that the integration for multi-value vectors has some nicer APIs to take advantage of (e.g. KnnCollector) and it could possibly copy/paste the deduplicating nearest neighbor min-heap implementation.
   
   > As a side note, do you happen to have any performance benchmark?
   
   The following test was completed over 139004 documents with 768 float32 dimensions.
   
   The statistics for the nested value distributions:
   
   `1944` total unique documents
   `62.0` median number of nested values
   `71.50411522633745` mean number of nested values
   `309` max number of nested values
   `1` min number of nested values
   `2156.9469722481676` variance of nested values
   
   ```
   |                                        50th percentile latency |          knn-search-10-100 |   3.10031     |     ms |
   |                                        90th percentile latency |          knn-search-10-100 |   3.5629      |     ms |
   |                                        99th percentile latency |          knn-search-10-100 |   4.60912     |     ms |
   |                                      99.9th percentile latency |          knn-search-10-100 |  14.322       |     ms |
   |                                       100th percentile latency |          knn-search-10-100 |  72.6463      |     ms |
   |                                        50th percentile latency |   knn-nested-search-10-100 |   6.2615      |     ms |
   |                                        90th percentile latency |   knn-nested-search-10-100 |   6.95849     |     ms |
   |                                        99th percentile latency |   knn-nested-search-10-100 |   7.8881      |     ms |
   |                                      99.9th percentile latency |   knn-nested-search-10-100 |  12.0871      |     ms |
   |                                       100th percentile latency |   knn-nested-search-10-100 |  57.9238      |     ms |
   |                                        50th percentile latency |        knn-search-100-1000 |   7.30288     |     ms |
   |                                        90th percentile latency |        knn-search-100-1000 |   8.18694     |     ms |
   |                                        99th percentile latency |        knn-search-100-1000 |   9.23673     |     ms |
   |                                      99.9th percentile latency |        knn-search-100-1000 |  18.7072      |     ms |
   |                                       100th percentile latency |        knn-search-100-1000 |  23.8712      |     ms |
   |                                        50th percentile latency | knn-search-nested-100-1000 |  26.6446      |     ms |
   |                                        90th percentile latency | knn-search-nested-100-1000 |  38.2561      |     ms |
   |                                        99th percentile latency | knn-search-nested-100-1000 |  44.3627      |     ms |
   |                                      99.9th percentile latency | knn-search-nested-100-1000 |  51.1843      |     ms |
   |                                       100th percentile latency | knn-search-nested-100-1000 |  52.0864      |     ms |
   ```
   
   GASP! Nested seems 2x to 4x slower!
   
   But, keep in mind, we are eagerly joining! When I dug into the difference, I discovered that eagerly joining on this dataset meant we were visiting 3x to 5x more vectors. Consequently doing 3-5x more vector comparisons and deeper exploration of the graph. This lines up really nicely with the performance difference.
   
   Since HNSW is `log(n)` the overall constant overhead of nested seems rather minor compared to the need to gather nearest vectors.
   
   I am not sure these numbers are reflective of other nested/block-joining operations (like a term search).
   
   


-- 
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-unsubscribe@lucene.apache.org

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


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


Re: [PR] Add ParentJoin KNN support [lucene]

Posted by "gauravj88 (via GitHub)" <gi...@apache.org>.
gauravj88 commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1857999310

   Hello @benwtrent @alessandrobenedetti , I'm seeking assistance in comprehending how to calculate the total score for a parent document, sort it in descending order by the score, and additionally, only compute matched child documents with it's score. Your guidance is appreciated. Here's the Solr query I've written.
   {!parent which=vector_status_s:P}{!knn f=vector topK=5}[-0.11524472,-0.015096951,-0.02416302,-0.042546727,-0.026223207,-0.024288718,-0.06648887,-0.019747224,-0.15247388,0.017357947,-0.07997921,-0.0044639846,0.026787976,0.04674327,-0.025140975,0.022736596,0.038513225,-0.007743583,0.008468372,-0.010870436,-0.049723275,0.052544124,-0.01040933,-0.021913795,0.055262197,0.03814301,-0.030052986,0.011873034,0.014196521,-0.045679808,-0.030343458,0.0027234245,0.067535505,-0.0065462803,0.014386235,0.0815663,0.034734566,0.076742254,-0.047816984,-0.06893713,-0.00575254,-0.0065407623,0.032530636,-0.047053427,-0.045342736,0.05458816,-0.0036792993,-0.119853236,0.0621119,0.05868241,-0.026309596,-0.07372244,0.098664746,-0.10785703,0.058371536,0.11598263,0.0035437504,0.034718044,-0.008448179,-0.015374862,0.11060915,-0.034994677,-0.07641869,-0.018799445,0.06742509,0.032640636,-0.0805842,0.056232687,-0.055578843,-0.09173681,0.055953458,-0.093306944,-0.005010136,-0.027816057,0.069614165,0.017868416,0.0
 64177155,0.033978168,0.0015766103,-0.024414165,-0.03577587,0.0456416,-0.02556829,-0.019146064,-0.061762888,0.0881951,0.021560378,-0.021047728,0.012747205,0.022434331,0.018475346,-0.046412993,-0.04435442,0.04168245,0.058627237,0.0135493,-0.06728198,0.07312625,-0.05581615,0.04042105,-0.011703705,0.08142669,-0.02410761,-0.091190875,-0.028388929,-0.032626763,-0.05041609,0.06420719,0.046886727,0.06463699,0.035878975,-0.001365671,-0.069236726,-0.06430995,0.03162415,-0.07464388,-0.03160973,0.04416939,-0.024402045,-0.014191367,0.07411146,0.104716584,-0.048191704,-0.005468206,0.03430066,2.9004048E-4,0.026272327,1.70109E-33,0.029819172,0.06773484,-0.00742313,0.071461275,0.07202007,0.020473966,0.038026482,0.08200619,0.02915628,-0.012804087,0.059027206,0.089257926,-0.06499567,-0.0025720217,-0.046283025,-0.07170174,-0.046748683,-0.112400465,-0.035749055,-0.0014416518,0.020358646,0.046174794,-0.052782748,0.058808744,0.054796845,-0.060931858,0.108469814,0.033284586,-0.03280937,0.024645116,-0.08053
 44,0.034969736,-0.032420795,0.004894371,-0.026090093,0.0041577136,-0.04494302,-0.037979875,-0.03771678,-0.02735791,-0.09234106,0.002481496,-0.07598019,-3.466485E-4,-0.06326391,-0.00995851,0.035185788,0.07036043,0.10276299,-0.059668124,-0.0083749425,0.018889284,0.027138462,0.07999683,-0.019039072,-0.061816998,0.017313186,-0.051152766,0.039306473,-0.06226915,0.01846972,0.04963292,0.020879248,0.14606093,-0.09989049,-0.005342353,0.06521556,-0.024311792,0.041433204,0.066303164,0.022143282,-0.035711862,0.009525745,-0.057548966,-0.0806604,0.0028130244,-0.010393038,0.112975076,-0.0018174571,0.030156288,-0.02206562,0.04948243,0.032909732,0.01555169,0.014569762,0.0039269393,0.017587842,-0.0105174305,0.016612422,0.077120796,-0.042344075,-0.06468686,-0.036919363,0.03040028,-0.019409828,-3.251894E-33,0.040625818,0.016581528,-0.021927938,0.021516323,0.03471849,-0.045520667,-0.017614665,-0.07006911,0.0071153007,0.0025327879,0.005648088,0.04241776,-0.010652379,-0.023858054,0.012913902,-0.027931714,
 0.004287367,-0.027264068,0.061210893,-0.039421324,0.12714641,0.03849749,-0.094134904,-0.007945886,0.04947471,-0.056016088,-0.087781586,0.0797169,-0.025210094,-0.06051712,-0.019571543,-0.060732212,-0.012113058,0.017380701,-0.06130845,0.06727084,-0.033202328,0.091200486,0.02490767,-0.07749138,0.05946457,-0.02085205,-0.0049607577,0.050158907,-0.024974424,-0.054097354,-0.034355577,-0.0655865,-0.06576413,0.056954406,0.05969364,0.08660039,0.0020785942,-0.09242396,-0.052793916,0.034658235,0.03479103,0.0055098752,0.00982389,0.11976097,0.03207969,-0.012991179,0.05993113,0.037464924,-0.05148737,0.0068404386,0.07629449,-0.0012071802,-0.05490101,0.061263554,-0.008928003,-0.004517294,0.03881187,-0.030366309,-0.032273546,0.0091122035,0.0014949276,0.031635325,-0.028407004,-0.069446735,0.036516335,-0.014874138,0.049066026,0.12900232,0.05830397,0.06482125,0.06983362,0.041088205,0.0348922,0.07437936,-0.011469576,-0.043531597,-0.009242795,-0.060591742,-0.11750616,-5.111815E-8,-0.015631394,-0.027901249
 ,-0.123293586,-0.05102494,-0.012230896,-0.0066172937,0.025514597,0.033083055,-0.022466997,0.059927735,0.09024767,-0.039916337,-0.09570077,0.0348283,0.020531913,-0.071214065,-0.023437353,0.013603563,-0.100605406,-0.067646325,0.031216368,0.0102580255,-0.017372983,0.031901423,-0.032343123,0.011557923,-0.019640395,0.0059259743,0.030634984,0.023833074,-0.082269505,0.0863291,-0.014290165,-0.027523914,-0.018137619,-0.030911706,0.01040259,0.01957091,-0.028549556,-0.04439533,-0.061055064,-0.005775817,0.025976641,0.008309728,-0.030016769,-0.025356589,-0.07005577,-0.02789409,0.028691122,-0.013100756,-0.018772617,0.037944615,0.0311197,0.009778543,0.0055453815,-0.01825943,-0.022618072,-0.07403188,0.03977728,-0.08186114,-0.0038845171,-0.054700762,0.0027615656,0.054946393] fl=*,[child]


-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274813661


##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected int visitedCount;
+  private final int visitLimit;

Review Comment:
   I can do that



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274841199


##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected int visitedCount;
+  private final int visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, int visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  public final boolean incomplete() {
+    return visitedCount >= visitLimit;
+  }
+
+  public final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited count
+   */
+  public final int visitedCount() {
+    return visitedCount;
+  }
+
+  public final int visitLimit() {
+    return visitLimit;
+  }
+
+  public final int k() {

Review Comment:
   OK I see it now, sorry for the noise.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1275036311


##########
lucene/core/src/java/org/apache/lucene/codecs/KnnVectorsReader.java:
##########
@@ -79,14 +80,13 @@ protected KnnVectorsReader() {}
    *
    * @param field the vector field to search
    * @param target the vector-valued query
-   * @param k the number of docs to return
+   * @param knnResults a KnnResults collector and relevant settings for gathering vector results
    * @param acceptDocs {@link Bits} that represents the allowed documents to match, or {@code null}
    *     if they are all allowed to match.
-   * @param visitedLimit the maximum number of nodes that the search is allowed to visit
    * @return the k nearest neighbor documents, along with their (similarity-specific) scores.
    */
   public abstract TopDocs search(
-      String field, float[] target, int k, Bits acceptDocs, int visitedLimit) throws IOException;
+      String field, float[] target, KnnResults knnResults, Bits acceptDocs) throws IOException;

Review Comment:
   Well, this may cause weirdness with Lucene90 :/ If this is a strongly held opinion, I can work on figuring out that searcher. 



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1275257249


##########
lucene/core/src/java/org/apache/lucene/search/KnnCollector.java:
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.search;
+
+/**
+ * KnnCollector is a knn collector used for gathering kNN results and providing topDocs from the
+ * gathered neighbors
+ */
+public interface KnnCollector {
+
+  /**
+   * If search visits too many documents, the results collector will terminate early. Usually, this
+   * is due to some restricted filter on the document set.
+   *
+   * <p>When collection is earlyTerminated, the results are not a correct representation of k
+   * nearest neighbors.
+   *
+   * @return is the current result set marked as incomplete?
+   */
+  boolean earlyTerminated();
+
+  /**
+   * @param count increments the visited vector count, must be greater than 0.
+   */
+  void incVisitedCount(int count);
+
+  /**
+   * @return the current visited vector count
+   */
+  long visitedCount();
+
+  /**
+   * @return the visited vector limit
+   */
+  long visitLimit();
+
+  /**
+   * @return the expected number of collected results
+   */
+  int k();
+
+  /**
+   * Collect the provided docId and include in the result set.
+   *
+   * @param docId of the vector to collect
+   * @param similarity its calculated similarity
+   * @return true if the vector is collected
+   */
+  boolean collect(int docId, float similarity);
+
+  /**
+   * This method is utilized during search to ensure only competitive results are explored.
+   *
+   * <p>Consequently, if this results collector wants to collect `k` results, this should return
+   * {@link Float#NEGATIVE_INFINITY} when not full.
+   *
+   * <p>When full, the minimum score should be returned.
+   *
+   * @return the current minimum competitive similarity in the collection
+   */
+  float minCompetitiveSimilarity();
+
+  /**
+   * This drains the collected nearest kNN results and returns them in a new {@link TopDocs}
+   * collection, ordered by score descending

Review Comment:
   will do!



##########
lucene/core/src/java/org/apache/lucene/search/AbstractKnnCollector.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.search;
+
+/**
+ * AbstractKnnCollector is the default implementation for a knn collector used for gathering kNN
+ * results and providing topDocs from the gathered neighbors
+ */
+public abstract class AbstractKnnCollector implements KnnCollector {
+
+  private long visitedCount;
+  private final long visitLimit;
+  private final int k;
+
+  protected AbstractKnnCollector(int k, long visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  /**
+   * If search visits too many documents, the results collector will terminate early. Usually, this
+   * is due to some restricted filter on the document set.
+   *
+   * <p>When collection is earlyTerminated, the results are not a correct representation of k
+   * nearest neighbors.
+   *
+   * @return is the current result set marked as incomplete?
+   */
+  @Override
+  public final boolean earlyTerminated() {
+    return visitedCount >= visitLimit;
+  }
+
+  /**
+   * @param count increments the visited vector count, must be greater than 0.
+   */
+  @Override
+  public final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited vector count
+   */
+  @Override
+  public final long visitedCount() {
+    return visitedCount;
+  }
+
+  /**
+   * @return the visited vector limit
+   */
+  @Override
+  public final long visitLimit() {
+    return visitLimit;
+  }
+
+  /**
+   * @return the expected number of collected results
+   */
+  @Override
+  public final int k() {
+    return k;
+  }
+
+  /**
+   * Collect the provided docId and include in the result set.
+   *
+   * @param docId of the vector to collect
+   * @param similarity its calculated similarity
+   * @return true if the vector is collected
+   */
+  @Override
+  public abstract boolean collect(int docId, float similarity);
+
+  /**
+   * This method is utilized during search to ensure only competitive results are explored.
+   *
+   * <p>Consequently, if this results collector wants to collect `k` results, this should return
+   * {@link Float#NEGATIVE_INFINITY} when not full.
+   *
+   * <p>When full, the minimum score should be returned.
+   *
+   * @return the current minimum competitive similarity in the collection
+   */
+  @Override
+  public abstract float minCompetitiveSimilarity();
+
+  /**
+   * This drains the collected nearest kNN results and returns them in a new {@link TopDocs}
+   * collection, ordered by score descending
+   *
+   * @return The collected top documents
+   */

Review Comment:
   Ah, yeah 🤦 



##########
lucene/core/src/java/org/apache/lucene/search/AbstractKnnCollector.java:
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.search;
+
+/**
+ * AbstractKnnCollector is the default implementation for a knn collector used for gathering kNN
+ * results and providing topDocs from the gathered neighbors
+ */
+public abstract class AbstractKnnCollector implements KnnCollector {
+
+  private long visitedCount;
+  private final long visitLimit;
+  private final int k;
+
+  protected AbstractKnnCollector(int k, long visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  /**
+   * If search visits too many documents, the results collector will terminate early. Usually, this
+   * is due to some restricted filter on the document set.
+   *
+   * <p>When collection is earlyTerminated, the results are not a correct representation of k
+   * nearest neighbors.
+   *
+   * @return is the current result set marked as incomplete?
+   */
+  @Override
+  public final boolean earlyTerminated() {
+    return visitedCount >= visitLimit;
+  }
+
+  /**
+   * @param count increments the visited vector count, must be greater than 0.
+   */
+  @Override
+  public final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited vector count
+   */
+  @Override
+  public final long visitedCount() {
+    return visitedCount;
+  }
+
+  /**
+   * @return the visited vector limit
+   */
+  @Override
+  public final long visitLimit() {
+    return visitLimit;
+  }
+
+  /**
+   * @return the expected number of collected results
+   */
+  @Override
+  public final int k() {
+    return k;
+  }
+
+  /**
+   * Collect the provided docId and include in the result set.
+   *
+   * @param docId of the vector to collect
+   * @param similarity its calculated similarity
+   * @return true if the vector is collected
+   */
+  @Override
+  public abstract boolean collect(int docId, float similarity);
+
+  /**
+   * This method is utilized during search to ensure only competitive results are explored.
+   *
+   * <p>Consequently, if this results collector wants to collect `k` results, this should return
+   * {@link Float#NEGATIVE_INFINITY} when not full.
+   *
+   * <p>When full, the minimum score should be returned.
+   *
+   * @return the current minimum competitive similarity in the collection
+   */
+  @Override
+  public abstract float minCompetitiveSimilarity();
+
+  /**
+   * This drains the collected nearest kNN results and returns them in a new {@link TopDocs}
+   * collection, ordered by score descending
+   *
+   * @return The collected top documents
+   */

Review Comment:
   Ah, yeah 🤦 



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274831373


##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected int visitedCount;
+  private final int visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, int visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  public final boolean incomplete() {
+    return visitedCount >= visitLimit;
+  }
+
+  public final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited count
+   */
+  public final int visitedCount() {
+    return visitedCount;
+  }
+
+  public final int visitLimit() {
+    return visitLimit;
+  }
+
+  public final int k() {

Review Comment:
   @jpountz 
   ```
               new NeighborQueue(knnResults.k(), true),
   ```
   This is in the 
   ```
   public static KnnResults search(
         byte[] query,
         KnnResults knnResults,
         RandomAccessVectorValues<byte[]> vectors,
         VectorEncoding vectorEncoding,
         VectorSimilarityFunction similarityFunction,
         HnswGraph graph,
         Bits acceptOrds)
   ```
   Method for both `float[]` and `byte[]`. Creating a `maxHeap` for keeping track of expected candidates



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274825776


##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected int visitedCount;
+  private final int visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, int visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  public final boolean incomplete() {
+    return visitedCount >= visitLimit;
+  }
+
+  public final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited count
+   */
+  public final int visitedCount() {
+    return visitedCount;
+  }
+
+  public final int visitLimit() {
+    return visitLimit;
+  }
+
+  public final int k() {

Review Comment:
   I'm not seeing where it's used to pre-allocate candidate collection? (To be clear, I'm only suggesting we remove it from the public `KnnResults` API, `TopKnnResults` would still take a `k` parameter in its ctor)



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1276001713


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/TopKnnCollector.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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;

Review Comment:
   Hmm, having to know what codec is used to pick the optimal collector sounds bad, codec changes should be transparent as far as the search logic is concerned and ideally only affect performance and recall. Even internally, 
    what would it mean for `LeafReader#searchNearestVectors(String, float[], int, Bits, int)` since it cannot know what codec is being used?



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on pull request #12434: Add ParentJoin KNN support

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1657050056

   I agree that there is similarity in that in both cases it boils down to whether or not you can accept having less than `k` hits. However the degradation is brutal with filtering as you either need to evaluate the filter across the entire segment to load it into a bitset (not great for both runtime (if the filter cardinality is high) and memory usage) or linearly scan all filter matches (not great either). Here the degradation is much more graceful as you only pay some overhead for vectors that get collected. For filtering, I could see a case for requesting k'>k vectors and then do post filtering. For this case I think I would always want to use this feature, potentially combined with the `visitLimit` option to protect against worst-case conditions like a million child docs per parent that would make collisions frequent.


-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1269789691


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResults.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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 org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  /** KnnResults when exiting search early and returning empty top docs */
+  static class EmptyKnnResults extends KnnResults {
+    public EmptyKnnResults(int visitedCount) {
+      this.visitedCount = visitedCount;
+    }
+
+    @Override
+    public void doClear() {}
+
+    @Override
+    public void collect(int vectorId, float similarity) {
+      throw new IllegalArgumentException();
+    }
+
+    @Override
+    public boolean collectWithOverflow(int vectorId, float similarity) {
+      return false;
+    }
+
+    @Override
+    public boolean isFull() {
+      return true;
+    }
+
+    @Override
+    public float minSimilarity() {
+      return 0;
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TotalHits th = new TotalHits(visitedCount, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO);
+      return new TopDocs(th, new ScoreDoc[0]);
+    }
+  }
+
+  protected int visitedCount;
+  private boolean incomplete;
+
+  final void clear() {
+    this.visitedCount = 0;
+    this.incomplete = false;
+    doClear();
+  }
+
+  /** Clear the current results. */
+  abstract void doClear();
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  final boolean incomplete() {
+    return incomplete;
+  }
+
+  /** Mark the current result set as incomplete */
+  final void markIncomplete() {
+    this.incomplete = true;
+  }
+
+  /**
+   * @param count set the current visited count to the provided value
+   */
+  final void setVisitedCount(int count) {
+    this.visitedCount = count;
+  }
+
+  /**
+   * @return the current visited count
+   */
+  final int visitedCount() {
+    return visitedCount;
+  }
+
+  /**
+   * Collect the provided vectorId and include in the result set.
+   *
+   * @param vectorId the vector to collect
+   * @param similarity its calculated similarity
+   */
+  abstract void collect(int vectorId, float similarity);
+
+  /**
+   * @param vectorId the vector to collect
+   * @param similarity its calculated similarity
+   * @return true if the vector is collected
+   */
+  abstract boolean collectWithOverflow(int vectorId, float similarity);

Review Comment:
   > we seem to be doing a collectWithOverflow when the result set is full, so maybe we could have a single collect method and automatically do the right thing depending on whether it's full or not?
   
   def I will see how this effects things.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1269794542


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResultsProvider.java:
##########
@@ -0,0 +1,25 @@
+/*
+ * 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;
+
+/** knn results provider */
+public interface KnnResultsProvider {
+  KnnResults getKnnResults(IntToIntFunction vectorToOrd);

Review Comment:
   > This vectorToOrd argument feels specific to the current implementation. Could the API work on doc IDs?
   
   Even regular TopK needs access to the ordinal->docId transformation because the vector storage could be sparse and eagerly transforming ordinal->docId would add unnecessary overhead. TopK only needs to do the transformation from ordinal->docId when it builds the topdocs result set. 
   
   ToParentJoin needs the docId to compare against the parent bitset. I can see if there is a better way to get the ord->doc transformation into ToParentJoin without hurting sparse vector search performance.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1273659464


##########
lucene/core/src/java/org/apache/lucene/util/hnsw/KnnResults.java:
##########
@@ -0,0 +1,175 @@
+/*
+ * 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 org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  /** KnnResults when exiting search early and returning empty top docs */
+  static class EmptyKnnResults extends KnnResults {
+    public EmptyKnnResults(int k, int visitedCount, int visitLimit) {
+      super(k, visitLimit);
+      this.visitedCount = visitedCount;
+    }
+
+    @Override
+    public void doClear() {}
+
+    @Override
+    public boolean collect(int vectorId, float similarity) {
+      throw new IllegalArgumentException();
+    }
+
+    @Override
+    public boolean isFull() {
+      return true;
+    }
+
+    @Override
+    public float minSimilarity() {
+      return 0;
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TotalHits th = new TotalHits(visitedCount, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO);
+      return new TopDocs(th, new ScoreDoc[0]);
+    }
+  }
+
+  static class OrdinalTranslatedKnnResults extends KnnResults {
+    private final KnnResults in;
+    private final IntToIntFunction vectorOrdinalToDocId;
+
+    OrdinalTranslatedKnnResults(KnnResults in, IntToIntFunction vectorOrdinalToDocId) {
+      super(in.k, in.visitLimit);
+      this.in = in;
+      this.vectorOrdinalToDocId = vectorOrdinalToDocId;
+    }
+
+    @Override
+    void doClear() {
+      in.clear();
+    }
+
+    @Override
+    boolean collect(int vectorId, float similarity) {
+      return in.collect(vectorOrdinalToDocId.apply(vectorId), similarity);
+    }
+
+    @Override
+    boolean isFull() {
+      return in.isFull();
+    }
+
+    @Override
+    float minSimilarity() {
+      return in.minSimilarity();
+    }
+
+    @Override
+    public TopDocs topDocs() {
+      TopDocs td = in.topDocs();
+      return new TopDocs(
+          new TotalHits(
+              visitedCount(),
+              incomplete()
+                  ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+                  : TotalHits.Relation.EQUAL_TO),
+          td.scoreDocs);
+    }
+  }
+
+  protected int visitedCount;
+  private final int visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, int visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  final void clear() {
+    this.visitedCount = 0;
+    doClear();
+  }
+
+  /** Clear the current results. */
+  abstract void doClear();
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  final boolean incomplete() {
+    return visitedCount >= visitLimit;
+  }
+
+  final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited count
+   */
+  final int visitedCount() {
+    return visitedCount;
+  }
+
+  final int visitLimit() {
+    return visitLimit;
+  }
+
+  public final int k() {
+    return k;
+  }
+
+  /**
+   * Collect the provided vectorId and include in the result set.
+   *
+   * @param vectorId the vector to collect
+   * @param similarity its calculated similarity
+   * @return true if the vector is collected
+   */
+  abstract boolean collect(int vectorId, float similarity);
+
+  /**
+   * @return Is the current result set considered full
+   */
+  abstract boolean isFull();

Review Comment:
   > and change call-sites to check if minSimilarity() returns a greater value than NEGATIVE_INFINITY, in order to keep the surface area minimal.
   
   This won't work in the builder. We only update the similarity candidate if we are "full" and accepted it via `collect`. Otherwise we don't know if the best minimal similarity was discovered. 
   
   I might be able to adjust this by always doing `minAcceptedSimilarity = Math.max(results.topScore(), minAcceptedSimilarity` if `collect` returns true.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274813058


##########
lucene/core/src/java/org/apache/lucene/search/KnnResults.java:
##########
@@ -0,0 +1,88 @@
+/*
+ * 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.search;
+
+/**
+ * KnnResults is a collector for gathering kNN results and providing topDocs from the gathered
+ * neighbors
+ */
+public abstract class KnnResults {
+
+  protected int visitedCount;
+  private final int visitLimit;
+  private final int k;
+
+  protected KnnResults(int k, int visitLimit) {
+    this.visitLimit = visitLimit;
+    this.k = k;
+  }
+
+  /**
+   * @return is the current result set marked as incomplete?
+   */
+  public final boolean incomplete() {
+    return visitedCount >= visitLimit;
+  }
+
+  public final void incVisitedCount(int count) {
+    assert count > 0;
+    this.visitedCount += count;
+  }
+
+  /**
+   * @return the current visited count
+   */
+  public final int visitedCount() {
+    return visitedCount;
+  }
+
+  public final int visitLimit() {
+    return visitLimit;
+  }
+
+  public final int k() {

Review Comment:
   @jpountz it allows us to pre-allocate candidate collection. If we don't pre-allocate, we will likely have more than one array copy and extension while collecting candidates. 
   
   We could call it something else, but I we should still have it.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1275050439


##########
lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java:
##########
@@ -278,7 +276,7 @@ public TopDocs search(String field, float[] target, int k, Bits acceptDocs, int
   }
 
   @Override
-  public TopDocs search(String field, byte[] target, int k, Bits acceptDocs, int visitedLimit)
+  public TopDocs search(String field, byte[] target, KnnResults knnResults, Bits acceptDocs)

Review Comment:
   > if we should collect entries from the result NeighborQueue into the KnnResults object.
   
   🤔 I could. This would then allow the change of the method definition to be `void` and we always just add things to the collector. 



##########
lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java:
##########
@@ -278,7 +276,7 @@ public TopDocs search(String field, float[] target, int k, Bits acceptDocs, int
   }
 
   @Override
-  public TopDocs search(String field, byte[] target, int k, Bits acceptDocs, int visitedLimit)
+  public TopDocs search(String field, byte[] target, KnnResults knnResults, Bits acceptDocs)

Review Comment:
   > if we should collect entries from the result NeighborQueue into the KnnResults object.
   
   🤔 I could. This would then allow the change of the method definition to be `void` and we always just add things to the collector. 



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1651725953

   @msokolov && @alessandrobenedetti pinging y'all as you will probably be the most interested in this change.
   
   @alessandrobenedetti the original design did take some inspiration from your multi-value vector work. However, benchmarking & testing required significant changes. For deduplicating parent docIds during search, the hashMap is now part of the queue instead of iterating a cache outside  the heap. This improved performance significantly.
   
   I would say this is how folks should represent multi-valued vectors when they require access to the matching passage or additional metadata. Otherwise, deep changes are required in the codec to attach arbitrary metadata to the vectors themselves, which seems like overkill to me when we already have `join`.
   
   This does not obviate the need for "true" multi-value vector support (e.g. for late-interaction models, or multi-value vectors that don't require metadata). This does lay some nice groundwork that can improve that implementation (a custom collector that can deduplicate vectors to a docId while searching).


-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1274930576


##########
lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene90/Lucene90HnswVectorsReader.java:
##########
@@ -278,7 +276,7 @@ public TopDocs search(String field, float[] target, int k, Bits acceptDocs, int
   }
 
   @Override
-  public TopDocs search(String field, byte[] target, int k, Bits acceptDocs, int visitedLimit)
+  public TopDocs search(String field, byte[] target, KnnResults knnResults, Bits acceptDocs)

Review Comment:
   >  I'm assuming it's fixable, but maybe this old codec makes it challenging?
   
   This old codec doesn't use the typical search path. Honestly, since new versions don't allow writing new documents to old codecs, I didn't see support for a new way to search as critical. 
   
   They would only need this change if a user indexed `join` documents into Lucene90 and haven't been able to deduplicate over parent doc id for the last year+. If that is the case, it seems weird that we haven't seen any issues related to 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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1654138467

   > One interesting side-effect of this change is that it gives us an entry point for supporting timeouts, I believe that we could do this via KnnCollector#earlyTerminated?
   
   🤔 yeah, it totally could if the collector has some signal indicating timeout, we can exit :)


-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on a diff in pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on code in PR #12434:
URL: https://github.com/apache/lucene/pull/12434#discussion_r1279441401


##########
lucene/join/src/java/org/apache/lucene/search/join/ToParentJoinKnnCollector.java:
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.search.join;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.apache.lucene.search.AbstractKnnCollector;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.search.TotalHits;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BitSet;
+
+/** parent joining knn collector, vector docIds are deduplicated according to the parent bit set. */
+class ToParentJoinKnnCollector extends AbstractKnnCollector {
+
+  private final BitSet parentBitSet;
+  private final NodeIdCachingHeap heap;
+
+  /**
+   * Create a new object for joining nearest child kNN documents with a parent bitset
+   *
+   * @param k The number of joined parent documents to collect
+   * @param visitLimit how many child vectors can be visited
+   * @param parentBitSet The leaf parent bitset
+   */
+  public ToParentJoinKnnCollector(int k, int visitLimit, BitSet parentBitSet) {
+    super(k, visitLimit);
+    this.parentBitSet = parentBitSet;
+    this.heap = new NodeIdCachingHeap(k);
+  }
+
+  /**
+   * If the heap is not full (size is less than the initialSize provided to the constructor), adds a
+   * new node-and-score element. If the heap is full, compares the score against the current top
+   * score, and replaces the top element if newScore is better than (greater than unless the heap is
+   * reversed), the current top score.
+   *
+   * <p>If docId's parent node has previously been collected and the provided nodeScore is less than
+   * the stored score it will not be collected.
+   *
+   * @param docId the neighbor docId
+   * @param nodeScore the score of the neighbor, relative to some other node
+   */
+  @Override
+  public boolean collect(int docId, float nodeScore) {
+    assert !parentBitSet.get(docId);
+    int nodeId = parentBitSet.nextSetBit(docId);
+    return heap.insertWithOverflow(nodeId, nodeScore);
+  }
+
+  @Override
+  public float minCompetitiveSimilarity() {
+    return heap.size >= k() ? heap.topScore() : Float.NEGATIVE_INFINITY;
+  }
+
+  @Override
+  public String toString() {
+    return "ToParentJoinKnnCollector[k=" + k() + ", size=" + heap.size() + "]";
+  }
+
+  @Override
+  public TopDocs topDocs() {
+    assert heap.size() <= k() : "Tried to collect more results than the maximum number allowed";
+    while (heap.size() > k()) {
+      heap.popToDrain();
+    }
+    ScoreDoc[] scoreDocs = new ScoreDoc[heap.size()];
+    for (int i = 1; i <= scoreDocs.length; i++) {
+      scoreDocs[scoreDocs.length - i] = new ScoreDoc(heap.topNode(), heap.topScore());
+      heap.popToDrain();
+    }
+
+    TotalHits.Relation relation =
+        earlyTerminated()
+            ? TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO
+            : TotalHits.Relation.EQUAL_TO;
+    return new TopDocs(new TotalHits(visitedCount(), relation), scoreDocs);
+  }
+
+  /**
+   * This is a minimum binary heap, inspired by {@link org.apache.lucene.util.LongHeap}. But instead
+   * of encoding and using `long` values. Node ids and scores are kept separate. Additionally, this
+   * prevents duplicate nodes from being added.
+   *
+   * <p>So, for every node added, we will update its score if the newly provided score is better.
+   * Every time we update a node's stored score, we ensure the heap's order.
+   */
+  private static class NodeIdCachingHeap {
+    private final int maxSize;
+    private int[] heapNodes;
+    private float[] heapScores;
+    private int size = 0;
+
+    // Used to keep track of nodeId -> positionInHeap. This way when new scores are added for a

Review Comment:
   @msokolov , how do you see a tuple including position working? The position is already known by the heap as it's the index of the node & score in the heap array. 
   
   What is required is a mapping of `nodeId -> heapIndex` to check if this node is already in the heap and where is it located. This way we can go to that particular heap index, update its score, and heapify.



-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] benwtrent commented on pull request #12434: Add ParentJoin KNN support

Posted by "benwtrent (via GitHub)" <gi...@apache.org>.
benwtrent commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1632517795

   > would it be enough or is there more?
   
   I will dig a bit more on making this cleaner. 
   
   My biggest performance concerns are around keeping track of the heap-index -> ID and shuffling those around so often and resolving the docId by vector ordinal on every push.


-- 
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-unsubscribe@lucene.apache.org

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


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


[GitHub] [lucene] jpountz commented on pull request #12434: Add ParentJoin KNN support

Posted by "jpountz (via GitHub)" <gi...@apache.org>.
jpountz commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1631586992

   From a quick look, this lower level KNN collection API looks interesting. It has currently a high surface - presumably because extending the queue was easier to have a working prototype, which is cool - I'm curious how much leaner it can be made. It feels like we'd need at least `collect(int docID, float similarity)`, `float minSimilarity()` and `TopDocs topDocs()`, would it be enough or is there more?


-- 
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-unsubscribe@lucene.apache.org

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


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


Re: [PR] Add ParentJoin KNN support [lucene]

Posted by "david-sitsky (via GitHub)" <gi...@apache.org>.
david-sitsky commented on PR #12434:
URL: https://github.com/apache/lucene/pull/12434#issuecomment-1835237536

   @benwtrent - did this really make it into 9.8.0?  I downloaded the 9.8.0 release and ToParentBlockJoinFloatKnnVectorQuery does not seem to be present.
   ```
   lucene-9.8.0/modules$ ls
   lucene-analysis-common-9.8.0.jar      lucene-codecs-9.8.0.jar       lucene-queries-9.8.0.jar
   lucene-analysis-icu-9.8.0.jar         lucene-core-9.8.0.jar         lucene-queryparser-9.8.0.jar
   lucene-analysis-kuromoji-9.8.0.jar    lucene-demo-9.8.0.jar         lucene-replicator-9.8.0.jar
   lucene-analysis-morfologik-9.8.0.jar  lucene-expressions-9.8.0.jar  lucene-sandbox-9.8.0.jar
   lucene-analysis-nori-9.8.0.jar        lucene-facet-9.8.0.jar        lucene-spatial3d-9.8.0.jar
   lucene-analysis-opennlp-9.8.0.jar     lucene-grouping-9.8.0.jar     lucene-spatial-extras-9.8.0.jar
   lucene-analysis-phonetic-9.8.0.jar    lucene-highlighter-9.8.0.jar  lucene-suggest-9.8.0.jar
   lucene-analysis-smartcn-9.8.0.jar     lucene-join-9.8.0.jar         META-INF
   lucene-analysis-stempel-9.8.0.jar     lucene-luke-9.8.0.jar         module-info.class
   lucene-backward-codecs-9.8.0.jar      lucene-memory-9.8.0.jar       org
   lucene-benchmark-9.8.0.jar            lucene-misc-9.8.0.jar
   lucene-classification-9.8.0.jar       lucene-monitor-9.8.0.jar
   lucene-9.8.0/modules$ for file in *.jar; do unzip -v $file | grep ToParentBlockJoinFloatKnnVectorQuery; done
   lucene-9.8.0/modules$ for file in *.jar; do unzip -v $file | grep ToParentBlockJoin; done
        948  Defl:N      541  43% 2023-09-21 21:59 3e2c2007  org/apache/lucene/search/join/ToParentBlockJoinQuery$1.class
       7806  Defl:N     3472  56% 2023-09-21 21:59 8e8db572  org/apache/lucene/search/join/ToParentBlockJoinQuery$BlockJoinScorer.class
       1814  Defl:N      709  61% 2023-09-21 21:59 d028b861  org/apache/lucene/search/join/ToParentBlockJoinQuery$BlockJoinWeight$1.class
       4114  Defl:N     1462  65% 2023-09-21 21:59 1e07024a  org/apache/lucene/search/join/ToParentBlockJoinQuery$BlockJoinWeight.class
       1589  Defl:N      838  47% 2023-09-21 21:59 cf93f84a  org/apache/lucene/search/join/ToParentBlockJoinQuery$ParentApproximation.class
       1869  Defl:N      836  55% 2023-09-21 21:59 1c0824bd  org/apache/lucene/search/join/ToParentBlockJoinQuery$ParentTwoPhase.class
       4723  Defl:N     1900  60% 2023-09-21 21:59 53b7ab29  org/apache/lucene/search/join/ToParentBlockJoinQuery.class
       2824  Defl:N     1085  62% 2023-09-21 21:59 323f6562  org/apache/lucene/search/join/ToParentBlockJoinSortField$1.class
       3191  Defl:N     1106  65% 2023-09-21 21:59 c9d8a699  org/apache/lucene/search/join/ToParentBlockJoinSortField$2$1.class
       1491  Defl:N      607  59% 2023-09-21 21:59 3a82677f  org/apache/lucene/search/join/ToParentBlockJoinSortField$2.class
       3196  Defl:N     1105  65% 2023-09-21 21:59 ec8017c1  org/apache/lucene/search/join/ToParentBlockJoinSortField$3$1.class
       1484  Defl:N      600  60% 2023-09-21 21:59 5bd0b2df  org/apache/lucene/search/join/ToParentBlockJoinSortField$3.class
       1368  Defl:N      576  58% 2023-09-21 21:59 29e61acb  org/apache/lucene/search/join/ToParentBlockJoinSortField$4$1$1.class
       3413  Defl:N     1152  66% 2023-09-21 21:59 4f73794f  org/apache/lucene/search/join/ToParentBlockJoinSortField$4$1.class
       1489  Defl:N      606  59% 2023-09-21 21:59 5132747c  org/apache/lucene/search/join/ToParentBlockJoinSortField$4.class
       1367  Defl:N      568  58% 2023-09-21 21:59 f6deee3a  org/apache/lucene/search/join/ToParentBlockJoinSortField$5$1$1.class
       3418  Defl:N     1151  66% 2023-09-21 21:59 d09c4733  org/apache/lucene/search/join/ToParentBlockJoinSortField$5$1.class
       1494  Defl:N      607  59% 2023-09-21 21:59 1e11e4cf  org/apache/lucene/search/join/ToParentBlockJoinSortField$5.class
       1266  Defl:N      685  46% 2023-09-21 21:59 18b34568  org/apache/lucene/search/join/ToParentBlockJoinSortField$6.class
       5837  Defl:N     2120  64% 2023-09-21 21:59 bfc259c3  org/apache/lucene/search/join/ToParentBlockJoinSortField.class
   ```


-- 
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-unsubscribe@lucene.apache.org

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


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