You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by "Adrien Grand (Jira)" <ji...@apache.org> on 2020/11/23 12:57:00 UTC

[jira] [Commented] (LUCENE-9619) Move Points from a visitor API to a custor-style API?

    [ https://issues.apache.org/jira/browse/LUCENE-9619?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17237347#comment-17237347 ] 

Adrien Grand commented on LUCENE-9619:
--------------------------------------

Here is what I have in mind in terms of API. The main downside compared to today is that it makes more assumptions about how points are implemented under the hood, e.g. it suggests a tree structure, which the current API doesn't. But I like that it would give us more control over how matching is performed as mentioned in the issue description. I still tried to push too many requirements to possible implementations, e.g. not enforcing an arity of 2 on inner nodes and not enforcing that the tree is balanced.

{code:java}
import java.io.IOException;
import java.util.function.IntConsumer;

import org.apache.lucene.search.DocIdSetIterator;

public abstract class PointValues {

  /* Global statistics, that don't change when moving from a node to another node. */

  /** Returns how many dimensions are represented in the values */
  public abstract int getNumDimensions() throws IOException;

  /** Returns how many dimensions are used for the index */
  public abstract int getNumIndexDimensions() throws IOException;

  /** Returns the number of bytes per dimension */
  public abstract int getBytesPerDimension() throws IOException;

  /** Return the total number of documents that have a value for this field, across all nodes. */
  public abstract int getTotalDocCount();

  /* Per-node statistics */
  
  /** Return the minimum packed value of the current node. */
  public abstract byte[] getMinPackedValue();

  /** Return the maximum packed value of the current node. */
  public abstract byte[] getMaxPackedValue();

  /** Return the total number of points under the current node. On the root node this returns the
   *  total number of points in the field on the current segment. */
  public abstract long size();

  /* API to walk the tree. */

  /** Move to the first child node and return {@code true} upon success. Returns {@code false} for
   *  leaf nodes and {@code true} otherwise. */
  public abstract boolean moveToChild();

  /** Move to the parent node and return {@code true} upon success. Returns {@code false} for the
   *  root node and {@code true} otherwise. */
  public abstract boolean moveToParent();

  /** Move to the next sibling node and return {@code true} upon success. Returns {@code false} if
   *  the current node has no more siblings. */
  public abstract boolean moveToSibling();

  /** A visitor for the content of the tree. */
  @FunctionalInterface
  public interface IntersectVisitor {

    /** Called for all documents in a leaf cell that crosses the query.  The consumer
     *  should scrutinize the packedValue to decide whether to accept it.  In the 1D case,
     *  values are visited in increasing order, and in the case of ties, in increasing
     *  docID order. */
    void visit(int docID, byte[] packedValue) throws IOException;

    /** Similar to {@link IntersectVisitor#visit(int, byte[])} but in this case the packedValue
     * can have more than one docID associated to it. The provided iterator should not escape the
     * scope of this method so that implementations of PointValues are free to reuse it,*/
    default void visit(DocIdSetIterator iterator, byte[] packedValue) throws IOException {
      int docID;
      while ((docID = iterator.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
        visit(docID, packedValue);
      }
    }
  }

  /** Visit all (document,value) pairs under the current node. {@link IntersectVisitor#visit} will
   *  be called {@link #size()} times. */
  public abstract void intersect(IntersectVisitor visitor);

  /** Visit all documents under the current node. {@link IntConsumer#accept} will be called
   *  {@link #size()} times. */
  public abstract void intersectAll(IntConsumer visitor);
}

{code}

Opinions?

> Move Points from a visitor API to a custor-style API?
> -----------------------------------------------------
>
>                 Key: LUCENE-9619
>                 URL: https://issues.apache.org/jira/browse/LUCENE-9619
>             Project: Lucene - Core
>          Issue Type: Improvement
>            Reporter: Adrien Grand
>            Priority: Minor
>
> Points' visitor API work well but there are a couple things we could make better if we moved to a cursor API, e.g.
>  - Term queries could return a DocIdSetIterator without having to materialize a BitSet.
>  - Nearest-neighbor search could work on top of the regular API instead of casting to BKDReader https://github.com/apache/lucene-solr/blob/6a7131ee246d700c2436a85ddc537575de2aeacf/lucene/sandbox/src/java/org/apache/lucene/sandbox/document/FloatPointNearestNeighbor.java#L296
>  - We could optimize counting the number of matches of a query by adding the number of points in a leaf without visiting documents where there are no deleted documents and a leaf fully matches the query.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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