You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2021/08/13 16:36:13 UTC

[GitHub] [lucene] msokolov opened a new pull request #241: LUCENE-10016: Added KnnVector index/query support to demo

msokolov opened a new pull request #241:
URL: https://github.com/apache/lucene/pull/241


   For some earlier discussion, see the first PR for this: https://github.com/apache/lucene/pull/238. This is just a rebased version of that one.


-- 
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 change in pull request #241: LUCENE-10016: Added KnnVector index/query support to demo

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #241:
URL: https://github.com/apache/lucene/pull/241#discussion_r689815437



##########
File path: lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java
##########
@@ -242,4 +266,54 @@ public static void doPagingSearch(
       }
     }
   }
+
+  private static Query addSemanticQuery(Query query, KnnVectorDict vectorDict) throws IOException {
+    StringBuilder semanticQueryText = new StringBuilder();
+    QueryFieldTermExtractor termExtractor = new QueryFieldTermExtractor("contents");
+    query.visit(termExtractor);
+    for (String term : termExtractor.terms) {
+      semanticQueryText.append(term).append(' ');

Review comment:
       Right, I agree that the analysis is not likely to be the same as we do when creating the Term-based Query. Personally I'm on the fence about whether it's crazy to demand the machine-learned model to re-learn basic text normalization, but apparently that is what people do, and it probably does allow some nuances to be captured better (there is information in letter case). But that's all beside the point. I'll abstract this Analyzer away behind a facade. 




-- 
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] mikemccand commented on a change in pull request #241: LUCENE-10016: Added KnnVector index/query support to demo

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #241:
URL: https://github.com/apache/lucene/pull/241#discussion_r689517013



##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorSimilarityFunction.java
##########
@@ -43,9 +43,9 @@ public float compare(float[] v1, float[] v2) {
   };
 
   /**
-   * If true, the scores associated with vector comparisons are in reverse order; that is, lower
-   * scores represent more similar vectors. Otherwise, if false, higher scores represent more
-   * similar vectors.
+   * If true, the scores associated with vector comparisons are nonnegative and in reverse order;
+   * that is, lower scores represent more similar vectors. Otherwise, if false, higher scores
+   * represent more similar vectors, and scores may be negative or positive.

Review comment:
       I love it when writing a demo/tutorial for a feature it also sometimes leads to improving the feature and/or its documentation too :)

##########
File path: lucene/demo/src/java/org/apache/lucene/demo/IndexFiles.java
##########
@@ -201,6 +235,18 @@ static void indexDoc(IndexWriter writer, Path file, long lastModified) throws IO
               "contents",
               new BufferedReader(new InputStreamReader(stream, StandardCharsets.UTF_8))));
 
+      if (dict != null) {
+        try (InputStream in = Files.newInputStream(file)) {
+          float[] vector =
+              new DemoKnnAnalyzer(dict)

Review comment:
       Could we create and re-use `DemoKnnAnalyzer(dict)` once in the ctor?

##########
File path: lucene/demo/src/java/org/apache/lucene/demo/IndexFiles.java
##########
@@ -126,8 +153,14 @@ public static void main(String[] args) {
       writer.close();
 
       Date end = new Date();
-      System.out.println(end.getTime() - start.getTime() + " total milliseconds");
-
+      try (IndexReader reader = DirectoryReader.open(dir)) {
+        System.out.println(
+            "Indexed "
+                + reader.numDocs()

Review comment:
       Thanks for improving this verbosity!

##########
File path: lucene/core/src/test/org/apache/lucene/search/TestKnnVectorQuery.java
##########
@@ -36,6 +36,7 @@
 /** TestKnnVectorQuery tests KnnVectorQuery. */
 public class TestKnnVectorQuery extends LuceneTestCase {
 
+  /** testEquals */

Review comment:
       Hmm this seems redundant?

##########
File path: lucene/demo/src/java/org/apache/lucene/demo/IndexFiles.java
##########
@@ -201,6 +235,18 @@ static void indexDoc(IndexWriter writer, Path file, long lastModified) throws IO
               "contents",
               new BufferedReader(new InputStreamReader(stream, StandardCharsets.UTF_8))));
 
+      if (dict != null) {

Review comment:
       Maybe rename `dict` to `knnDict` or `vectorDict` or so?  It's a bit too generic a name :)  Makes me think of "terms dict"!

##########
File path: lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java
##########
@@ -242,4 +266,54 @@ public static void doPagingSearch(
       }
     }
   }
+
+  private static Query addSemanticQuery(Query query, KnnVectorDict vectorDict) throws IOException {
+    StringBuilder semanticQueryText = new StringBuilder();
+    QueryFieldTermExtractor termExtractor = new QueryFieldTermExtractor("contents");
+    query.visit(termExtractor);
+    for (String term : termExtractor.terms) {
+      semanticQueryText.append(term).append(' ');
+    }
+    if (semanticQueryText.length() > 0) {

Review comment:
       Hmm I wonder what happens if the sole query term is the empty string :)

##########
File path: lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java
##########
@@ -242,4 +266,54 @@ public static void doPagingSearch(
       }
     }
   }
+
+  private static Query addSemanticQuery(Query query, KnnVectorDict vectorDict) throws IOException {
+    StringBuilder semanticQueryText = new StringBuilder();
+    QueryFieldTermExtractor termExtractor = new QueryFieldTermExtractor("contents");
+    query.visit(termExtractor);
+    for (String term : termExtractor.terms) {
+      semanticQueryText.append(term).append(' ');

Review comment:
       It's a bit weird that we pull all terms out of the `Query`, join them with whitespace, and then analyze them again (which splits on whitespace again).  Maybe we need (later) some sugar API that takes a `Query` and goes straight to a vector or something?

##########
File path: lucene/core/src/java/org/apache/lucene/util/VectorUtil.java
##########
@@ -117,17 +117,44 @@ public static float squareDistance(float[] v1, float[] v2) {
    * thrown for zero vectors.
    */
   public static void l2normalize(float[] v) {
+    l2normalize(v, true);
+  }
+
+  /**
+   * Modifies the argument to be unit length, dividing by its l2-norm.
+   *
+   * @param v the vector to normalize
+   * @param throwOnZero whether to throw an exception when <code>v</code> has all zeros
+   * @throws IllegalArgumentException when the vector is all zero and throwOnZero is true
+   */
+  public static void l2normalize(float[] v, boolean throwOnZero) {

Review comment:
       Same here.

##########
File path: lucene/demo/src/java/org/apache/lucene/demo/knn/KnnVectorDict.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.demo.knn;
+
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.FloatBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.regex.Pattern;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRefBuilder;
+import org.apache.lucene.util.VectorUtil;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.FSTCompiler;
+import org.apache.lucene.util.fst.PositiveIntOutputs;
+import org.apache.lucene.util.fst.Util;
+
+/**
+ * Manages a map from token to numeric vector for use with KnnVector indexing and search. The map is
+ * stored as an FST: token-to-ordinal plus a dense binary file holding the vectors.
+ */
+public class KnnVectorDict implements AutoCloseable {
+
+  private final FST<Long> fst;
+  private final FileChannel vectors;
+  private final ByteBuffer vbuffer;
+  private final int dimension;
+  private final byte[] output;
+
+  /**
+   * sole constructor
+   *
+   * @param knnDictPath the path where the KnnVectorDict is stored
+   */
+  public KnnVectorDict(Path knnDictPath) throws IOException {
+    String dictName = knnDictPath.getFileName().toString();
+    Path fstPath = knnDictPath.resolveSibling(dictName + ".fst");
+    Path binPath = knnDictPath.resolveSibling(dictName + ".bin");
+    fst = FST.read(fstPath, PositiveIntOutputs.getSingleton());
+    vectors = FileChannel.open(binPath);
+    long size = vectors.size();
+    if (size > Integer.MAX_VALUE) {
+      throw new IllegalArgumentException("vector file is too large: " + size + " bytes");
+    }
+    vbuffer = vectors.map(FileChannel.MapMode.READ_ONLY, 0, size);
+    dimension = vbuffer.getInt((int) (size - Integer.BYTES));
+    output = new byte[dimension * Float.BYTES];
+    if ((size - Integer.BYTES) % (dimension * Float.BYTES) != 0) {
+      throw new IllegalStateException(
+          "vector file size is not consonant with the vector dimension");
+    }
+  }
+
+  /**
+   * Get the vector corresponding to the given token. NOTE: the returned array is shared and its
+   * contents will be overwritten by subsequent calls. The caller is responsible to copy the data as
+   * needed.
+   *
+   * @param token the token to look up
+   * @return the (shared) vector corresponding to the token or null if the token was not present
+   */
+  public byte[] get(BytesRef token) throws IOException {
+    Long ord = Util.get(fst, token);
+    if (ord == null) {
+      return null;
+    }
+    vbuffer.position((int) (ord * dimension * Float.BYTES));
+    vbuffer.get(output);
+    return output;

Review comment:
       This reuse of `output` it sort of scary from thread safety standpoint?  I know `IndexFiles` doesn't use threads ... but if someone were to copy / paste this impl into their code, the thread safety would bite them.  E.g. you cannot share this `knnDict` in an analyzer if multiple threads might analyze at once.
   
   Maybe you could require caller to "own" the reusing, requiring them to pass in the byte[] vector?  And requiring that it is exactly the right size ...

##########
File path: lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java
##########
@@ -242,4 +266,54 @@ public static void doPagingSearch(
       }
     }
   }
+
+  private static Query addSemanticQuery(Query query, KnnVectorDict vectorDict) throws IOException {
+    StringBuilder semanticQueryText = new StringBuilder();
+    QueryFieldTermExtractor termExtractor = new QueryFieldTermExtractor("contents");
+    query.visit(termExtractor);
+    for (String term : termExtractor.terms) {
+      semanticQueryText.append(term).append(' ');
+    }
+    if (semanticQueryText.length() > 0) {
+      KnnVectorQuery knnQuery =
+          new KnnVectorQuery(
+              "contents-vector",
+              new DemoKnnAnalyzer(vectorDict).analyze("text", semanticQueryText.toString()),
+              1);
+      BooleanQuery.Builder builder = new BooleanQuery.Builder();
+      builder.add(query, BooleanClause.Occur.SHOULD);
+      builder.add(knnQuery, BooleanClause.Occur.SHOULD);
+      return builder.build();
+    }
+    return query;
+  }
+
+  private static class QueryFieldTermExtractor extends QueryVisitor {

Review comment:
       This seems like a separately useful utility class ... maybe we can later trot it out in a very `public` manner, if it isn't already available somewhere.

##########
File path: lucene/demo/src/test/org/apache/lucene/demo/TestDemo.java
##########
@@ -53,4 +55,64 @@ public void testIndexSearch() throws Exception {
     testOneSearch(indexDir, "derivative", 8);
     testOneSearch(indexDir, "license", 13);
   }
+
+  private void testVectorSearch(Path indexPath, String query, int expectedHitCount)
+      throws Exception {
+    testVectorSearch(indexPath, query, expectedHitCount, expectedHitCount);
+  }
+
+  private void testVectorSearch(
+      Path indexPath, String query, int expectedMinHitCount, int expectedMaxHitCount)
+      throws Exception {
+    PrintStream outSave = System.out;
+    try {
+      ByteArrayOutputStream bytes = new ByteArrayOutputStream();
+      PrintStream fakeSystemOut = new PrintStream(bytes, false, Charset.defaultCharset());
+      System.setOut(fakeSystemOut);
+      SearchFiles.main(
+          new String[] {
+            "-query", query, "-index", indexPath.toString(), "-semantic", "-paging", "20"
+          });
+      fakeSystemOut.flush();
+      String output =
+          bytes.toString(Charset.defaultCharset()); // intentionally use default encoding
+      int offset = output.indexOf(" total matching documents");
+      int hitCount =
+          Integer.parseInt(output.substring(output.lastIndexOf('\n', offset) + 1, offset));
+      assertTrue(
+          "unexpected hit count " + hitCount + " for query: " + query,
+          hitCount >= expectedMinHitCount && hitCount <= expectedMaxHitCount);
+    } finally {
+      System.setOut(outSave);
+    }
+  }
+
+  public void testKnnVectorSearch() throws Exception {
+    Path dir = getDataPath("test-files/docs");
+    Path indexDir = createTempDir("ContribDemoTest");
+    Path dictPath = indexDir.resolve("knn-dict");
+    Path vectorDictSource = getDataPath("test-files/knn-dict").resolve("knn-token-vectors");
+    KnnVectorDict.build(vectorDictSource, dictPath);
+
+    IndexFiles.main(
+        new String[] {
+          "-create",
+          "-docs",
+          dir.toString(),
+          "-index",
+          indexDir.toString(),
+          "-knn-dict",
+          dictPath.toString()
+        });
+    // These term-based matches are usually also the best semantic matches, but sometimes
+    // the vector search picks a different top hit
+    testVectorSearch(indexDir, "apache", 3, 4);
+    testVectorSearch(indexDir, "gnu", 6, 7);
+    testVectorSearch(indexDir, "derivative", 8, 9);
+    testVectorSearch(indexDir, "patent", 9, 10);
+    testVectorSearch(indexDir, "license", 13, 14);
+
+    // this matched 0 by token; semantic matching always adds one

Review comment:
       Hmm why does semantic matching always add one exactly?  Are we passing a `k=1` somewhere?
   
   Edit: OK I found it!  We are passing `1` when we create `KnnVectorQuery` -- maybe say that is the reason why it always adds one?

##########
File path: lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java
##########
@@ -54,42 +63,54 @@ public static void main(String[] args) throws Exception {
     String queries = null;
     int repeat = 0;
     boolean raw = false;
+    boolean semantic = false;
     String queryString = null;
     int hitsPerPage = 10;
 
     for (int i = 0; i < args.length; i++) {
-      if ("-index".equals(args[i])) {
-        index = args[i + 1];
-        i++;
-      } else if ("-field".equals(args[i])) {
-        field = args[i + 1];
-        i++;
-      } else if ("-queries".equals(args[i])) {
-        queries = args[i + 1];
-        i++;
-      } else if ("-query".equals(args[i])) {
-        queryString = args[i + 1];
-        i++;
-      } else if ("-repeat".equals(args[i])) {
-        repeat = Integer.parseInt(args[i + 1]);
-        i++;
-      } else if ("-raw".equals(args[i])) {
-        raw = true;
-      } else if ("-paging".equals(args[i])) {
-        hitsPerPage = Integer.parseInt(args[i + 1]);
-        if (hitsPerPage <= 0) {
-          System.err.println("There must be at least 1 hit per page.");
+      switch (args[i]) {
+        case "-index":
+          index = args[++i];
+          break;
+        case "-field":
+          field = args[++i];
+          break;
+        case "-queries":
+          queries = args[++i];
+          break;
+        case "-query":
+          queryString = args[++i];
+          break;
+        case "-repeat":
+          repeat = Integer.parseInt(args[++i]);
+          break;
+        case "-raw":
+          raw = true;
+          break;
+        case "-paging":
+          hitsPerPage = Integer.parseInt(args[++i]);
+          if (hitsPerPage <= 0) {
+            System.err.println("There must be at least 1 hit per page.");
+            System.exit(1);
+          }
+          break;
+        case "-semantic":

Review comment:
       Hmm do we use the term `semantic` elsewhere in the KNN code/APIs/docs?  Maybe rename to `-knn_vectors` or something?
   
   Also: can you fix the `Usage: ...` output above to include this new option too?

##########
File path: lucene/demo/src/java/org/apache/lucene/demo/knn/KnnVectorDict.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.demo.knn;
+
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.FloatBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.regex.Pattern;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRefBuilder;
+import org.apache.lucene.util.VectorUtil;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.FSTCompiler;
+import org.apache.lucene.util.fst.PositiveIntOutputs;
+import org.apache.lucene.util.fst.Util;
+
+/**
+ * Manages a map from token to numeric vector for use with KnnVector indexing and search. The map is
+ * stored as an FST: token-to-ordinal plus a dense binary file holding the vectors.
+ */
+public class KnnVectorDict implements AutoCloseable {
+
+  private final FST<Long> fst;
+  private final FileChannel vectors;
+  private final ByteBuffer vbuffer;
+  private final int dimension;
+  private final byte[] output;
+
+  /**
+   * sole constructor
+   *
+   * @param knnDictPath the path where the KnnVectorDict is stored
+   */
+  public KnnVectorDict(Path knnDictPath) throws IOException {
+    String dictName = knnDictPath.getFileName().toString();
+    Path fstPath = knnDictPath.resolveSibling(dictName + ".fst");
+    Path binPath = knnDictPath.resolveSibling(dictName + ".bin");
+    fst = FST.read(fstPath, PositiveIntOutputs.getSingleton());
+    vectors = FileChannel.open(binPath);
+    long size = vectors.size();
+    if (size > Integer.MAX_VALUE) {
+      throw new IllegalArgumentException("vector file is too large: " + size + " bytes");
+    }
+    vbuffer = vectors.map(FileChannel.MapMode.READ_ONLY, 0, size);
+    dimension = vbuffer.getInt((int) (size - Integer.BYTES));
+    output = new byte[dimension * Float.BYTES];
+    if ((size - Integer.BYTES) % (dimension * Float.BYTES) != 0) {
+      throw new IllegalStateException(
+          "vector file size is not consonant with the vector dimension");

Review comment:
       Whoa, I had never heard this `consonant with`, and I checked Google and sure enough it means what you are using it as here!
   
   Maybe include the size and dimension in the exception message?

##########
File path: lucene/demo/src/java/org/apache/lucene/demo/knn/KnnVectorDict.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.demo.knn;
+
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.FloatBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.regex.Pattern;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRefBuilder;
+import org.apache.lucene.util.VectorUtil;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.FSTCompiler;
+import org.apache.lucene.util.fst.PositiveIntOutputs;
+import org.apache.lucene.util.fst.Util;
+
+/**
+ * Manages a map from token to numeric vector for use with KnnVector indexing and search. The map is
+ * stored as an FST: token-to-ordinal plus a dense binary file holding the vectors.
+ */
+public class KnnVectorDict implements AutoCloseable {
+
+  private final FST<Long> fst;
+  private final FileChannel vectors;
+  private final ByteBuffer vbuffer;
+  private final int dimension;
+  private final byte[] output;
+
+  /**
+   * sole constructor
+   *
+   * @param knnDictPath the path where the KnnVectorDict is stored

Review comment:
       Maybe also document that it is actually two sibling files, `foobar.bin` and `foobar.fst`?




-- 
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 merged pull request #241: LUCENE-10016: Added KnnVector index/query support to demo

Posted by GitBox <gi...@apache.org>.
msokolov merged pull request #241:
URL: https://github.com/apache/lucene/pull/241


   


-- 
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 change in pull request #241: LUCENE-10016: Added KnnVector index/query support to demo

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #241:
URL: https://github.com/apache/lucene/pull/241#discussion_r689807176



##########
File path: lucene/core/src/test/org/apache/lucene/search/TestKnnVectorQuery.java
##########
@@ -36,6 +36,7 @@
 /** TestKnnVectorQuery tests KnnVectorQuery. */
 public class TestKnnVectorQuery extends LuceneTestCase {
 
+  /** testEquals */

Review comment:
       I added it because I was bedeviled by Javadoc build failures that later mysteriously cleared themselves up. It can safely be removed now (I hope!)




-- 
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 #241: LUCENE-10016: Added KnnVector index/query support to demo

Posted by GitBox <gi...@apache.org>.
msokolov commented on pull request #241:
URL: https://github.com/apache/lucene/pull/241#issuecomment-899917516


   Thanks for all the great comments! I incorporated changes to address and think the demo will be much clearer for 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] msokolov commented on a change in pull request #241: LUCENE-10016: Added KnnVector index/query support to demo

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #241:
URL: https://github.com/apache/lucene/pull/241#discussion_r689807080



##########
File path: lucene/core/src/java/org/apache/lucene/index/VectorSimilarityFunction.java
##########
@@ -43,9 +43,9 @@ public float compare(float[] v1, float[] v2) {
   };
 
   /**
-   * If true, the scores associated with vector comparisons are in reverse order; that is, lower
-   * scores represent more similar vectors. Otherwise, if false, higher scores represent more
-   * similar vectors.
+   * If true, the scores associated with vector comparisons are nonnegative and in reverse order;
+   * that is, lower scores represent more similar vectors. Otherwise, if false, higher scores
+   * represent more similar vectors, and scores may be negative or positive.

Review comment:
       Yeah! This exercise exposed a flaw with our previous conversion of KNN scores to Lucene Query result document scores.




-- 
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 change in pull request #241: LUCENE-10016: Added KnnVector index/query support to demo

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #241:
URL: https://github.com/apache/lucene/pull/241#discussion_r689815756



##########
File path: lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java
##########
@@ -242,4 +266,54 @@ public static void doPagingSearch(
       }
     }
   }
+
+  private static Query addSemanticQuery(Query query, KnnVectorDict vectorDict) throws IOException {
+    StringBuilder semanticQueryText = new StringBuilder();
+    QueryFieldTermExtractor termExtractor = new QueryFieldTermExtractor("contents");
+    query.visit(termExtractor);
+    for (String term : termExtractor.terms) {
+      semanticQueryText.append(term).append(' ');
+    }
+    if (semanticQueryText.length() > 0) {
+      KnnVectorQuery knnQuery =
+          new KnnVectorQuery(
+              "contents-vector",
+              new DemoKnnAnalyzer(vectorDict).analyze("text", semanticQueryText.toString()),
+              1);
+      BooleanQuery.Builder builder = new BooleanQuery.Builder();
+      builder.add(query, BooleanClause.Occur.SHOULD);
+      builder.add(knnQuery, BooleanClause.Occur.SHOULD);
+      return builder.build();
+    }
+    return query;
+  }
+
+  private static class QueryFieldTermExtractor extends QueryVisitor {

Review comment:
       I extracted this from the more sophisticated `o.a.l.search.highlight.QueryTermExtractor` since that did a lot I didn't need here, and using it required adding `highlight` as a dependency. Not sure what would be useful for others in a broader context; let's come back to it later?




-- 
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 change in pull request #241: LUCENE-10016: Added KnnVector index/query support to demo

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #241:
URL: https://github.com/apache/lucene/pull/241#discussion_r690447463



##########
File path: lucene/demo/src/test/org/apache/lucene/demo/TestDemo.java
##########
@@ -53,4 +55,64 @@ public void testIndexSearch() throws Exception {
     testOneSearch(indexDir, "derivative", 8);
     testOneSearch(indexDir, "license", 13);
   }
+
+  private void testVectorSearch(Path indexPath, String query, int expectedHitCount)
+      throws Exception {
+    testVectorSearch(indexPath, query, expectedHitCount, expectedHitCount);
+  }
+
+  private void testVectorSearch(
+      Path indexPath, String query, int expectedMinHitCount, int expectedMaxHitCount)
+      throws Exception {
+    PrintStream outSave = System.out;
+    try {
+      ByteArrayOutputStream bytes = new ByteArrayOutputStream();
+      PrintStream fakeSystemOut = new PrintStream(bytes, false, Charset.defaultCharset());
+      System.setOut(fakeSystemOut);
+      SearchFiles.main(
+          new String[] {
+            "-query", query, "-index", indexPath.toString(), "-semantic", "-paging", "20"
+          });
+      fakeSystemOut.flush();
+      String output =
+          bytes.toString(Charset.defaultCharset()); // intentionally use default encoding
+      int offset = output.indexOf(" total matching documents");
+      int hitCount =
+          Integer.parseInt(output.substring(output.lastIndexOf('\n', offset) + 1, offset));
+      assertTrue(
+          "unexpected hit count " + hitCount + " for query: " + query,
+          hitCount >= expectedMinHitCount && hitCount <= expectedMaxHitCount);
+    } finally {
+      System.setOut(outSave);
+    }
+  }
+
+  public void testKnnVectorSearch() throws Exception {
+    Path dir = getDataPath("test-files/docs");
+    Path indexDir = createTempDir("ContribDemoTest");
+    Path dictPath = indexDir.resolve("knn-dict");
+    Path vectorDictSource = getDataPath("test-files/knn-dict").resolve("knn-token-vectors");
+    KnnVectorDict.build(vectorDictSource, dictPath);
+
+    IndexFiles.main(
+        new String[] {
+          "-create",
+          "-docs",
+          dir.toString(),
+          "-index",
+          indexDir.toString(),
+          "-knn-dict",
+          dictPath.toString()
+        });
+    // These term-based matches are usually also the best semantic matches, but sometimes
+    // the vector search picks a different top hit
+    testVectorSearch(indexDir, "apache", 3, 4);
+    testVectorSearch(indexDir, "gnu", 6, 7);
+    testVectorSearch(indexDir, "derivative", 8, 9);
+    testVectorSearch(indexDir, "patent", 9, 10);
+    testVectorSearch(indexDir, "license", 13, 14);
+
+    // this matched 0 by token; semantic matching always adds one

Review comment:
       added some explanation in the comments




-- 
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 change in pull request #241: LUCENE-10016: Added KnnVector index/query support to demo

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #241:
URL: https://github.com/apache/lucene/pull/241#discussion_r689904039



##########
File path: lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java
##########
@@ -242,4 +266,54 @@ public static void doPagingSearch(
       }
     }
   }
+
+  private static Query addSemanticQuery(Query query, KnnVectorDict vectorDict) throws IOException {
+    StringBuilder semanticQueryText = new StringBuilder();
+    QueryFieldTermExtractor termExtractor = new QueryFieldTermExtractor("contents");
+    query.visit(termExtractor);
+    for (String term : termExtractor.terms) {
+      semanticQueryText.append(term).append(' ');
+    }
+    if (semanticQueryText.length() > 0) {
+      KnnVectorQuery knnQuery =
+          new KnnVectorQuery(
+              "contents-vector",
+              new DemoKnnAnalyzer(vectorDict).analyze("text", semanticQueryText.toString()),
+              1);

Review comment:
       +1 I'll enable this option, but I think still use 1 in the 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] msokolov commented on a change in pull request #241: LUCENE-10016: Added KnnVector index/query support to demo

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #241:
URL: https://github.com/apache/lucene/pull/241#discussion_r689827096



##########
File path: lucene/demo/src/java/org/apache/lucene/demo/knn/KnnVectorDict.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.demo.knn;
+
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.FloatBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.regex.Pattern;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRefBuilder;
+import org.apache.lucene.util.VectorUtil;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.FSTCompiler;
+import org.apache.lucene.util.fst.PositiveIntOutputs;
+import org.apache.lucene.util.fst.Util;
+
+/**
+ * Manages a map from token to numeric vector for use with KnnVector indexing and search. The map is
+ * stored as an FST: token-to-ordinal plus a dense binary file holding the vectors.
+ */
+public class KnnVectorDict implements AutoCloseable {
+
+  private final FST<Long> fst;
+  private final FileChannel vectors;
+  private final ByteBuffer vbuffer;
+  private final int dimension;
+  private final byte[] output;
+
+  /**
+   * sole constructor
+   *
+   * @param knnDictPath the path where the KnnVectorDict is stored
+   */
+  public KnnVectorDict(Path knnDictPath) throws IOException {
+    String dictName = knnDictPath.getFileName().toString();
+    Path fstPath = knnDictPath.resolveSibling(dictName + ".fst");
+    Path binPath = knnDictPath.resolveSibling(dictName + ".bin");
+    fst = FST.read(fstPath, PositiveIntOutputs.getSingleton());
+    vectors = FileChannel.open(binPath);
+    long size = vectors.size();
+    if (size > Integer.MAX_VALUE) {
+      throw new IllegalArgumentException("vector file is too large: " + size + " bytes");
+    }
+    vbuffer = vectors.map(FileChannel.MapMode.READ_ONLY, 0, size);
+    dimension = vbuffer.getInt((int) (size - Integer.BYTES));
+    output = new byte[dimension * Float.BYTES];
+    if ((size - Integer.BYTES) % (dimension * Float.BYTES) != 0) {
+      throw new IllegalStateException(
+          "vector file size is not consonant with the vector dimension");

Review comment:
       Sure. I liked this word since it is accurate but imprecise enough to avoid having to explain the relationship. Considered *congruent* but it has confusing associations.




-- 
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 change in pull request #241: LUCENE-10016: Added KnnVector index/query support to demo

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #241:
URL: https://github.com/apache/lucene/pull/241#discussion_r689903690



##########
File path: lucene/demo/src/java/org/apache/lucene/demo/knn/KnnVectorDict.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.demo.knn;
+
+import java.io.BufferedReader;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.FloatBuffer;
+import java.nio.channels.FileChannel;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.regex.Pattern;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IntsRefBuilder;
+import org.apache.lucene.util.VectorUtil;
+import org.apache.lucene.util.fst.FST;
+import org.apache.lucene.util.fst.FSTCompiler;
+import org.apache.lucene.util.fst.PositiveIntOutputs;
+import org.apache.lucene.util.fst.Util;
+
+/**
+ * Manages a map from token to numeric vector for use with KnnVector indexing and search. The map is
+ * stored as an FST: token-to-ordinal plus a dense binary file holding the vectors.
+ */
+public class KnnVectorDict implements AutoCloseable {
+
+  private final FST<Long> fst;
+  private final FileChannel vectors;
+  private final ByteBuffer vbuffer;
+  private final int dimension;
+  private final byte[] output;
+
+  /**
+   * sole constructor
+   *
+   * @param knnDictPath the path where the KnnVectorDict is stored
+   */
+  public KnnVectorDict(Path knnDictPath) throws IOException {
+    String dictName = knnDictPath.getFileName().toString();
+    Path fstPath = knnDictPath.resolveSibling(dictName + ".fst");
+    Path binPath = knnDictPath.resolveSibling(dictName + ".bin");
+    fst = FST.read(fstPath, PositiveIntOutputs.getSingleton());
+    vectors = FileChannel.open(binPath);
+    long size = vectors.size();
+    if (size > Integer.MAX_VALUE) {
+      throw new IllegalArgumentException("vector file is too large: " + size + " bytes");
+    }
+    vbuffer = vectors.map(FileChannel.MapMode.READ_ONLY, 0, size);
+    dimension = vbuffer.getInt((int) (size - Integer.BYTES));
+    output = new byte[dimension * Float.BYTES];
+    if ((size - Integer.BYTES) % (dimension * Float.BYTES) != 0) {
+      throw new IllegalStateException(
+          "vector file size is not consonant with the vector dimension");
+    }
+  }
+
+  /**
+   * Get the vector corresponding to the given token. NOTE: the returned array is shared and its
+   * contents will be overwritten by subsequent calls. The caller is responsible to copy the data as
+   * needed.
+   *
+   * @param token the token to look up
+   * @return the (shared) vector corresponding to the token or null if the token was not present
+   */
+  public byte[] get(BytesRef token) throws IOException {
+    Long ord = Util.get(fst, token);
+    if (ord == null) {
+      return null;
+    }
+    vbuffer.position((int) (ord * dimension * Float.BYTES));
+    vbuffer.get(output);
+    return output;

Review comment:
       Yeah, fair. I will switch to caller-supplied buffer




-- 
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 change in pull request #241: LUCENE-10016: Added KnnVector index/query support to demo

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #241:
URL: https://github.com/apache/lucene/pull/241#discussion_r689822979



##########
File path: lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java
##########
@@ -242,4 +266,54 @@ public static void doPagingSearch(
       }
     }
   }
+
+  private static Query addSemanticQuery(Query query, KnnVectorDict vectorDict) throws IOException {
+    StringBuilder semanticQueryText = new StringBuilder();
+    QueryFieldTermExtractor termExtractor = new QueryFieldTermExtractor("contents");
+    query.visit(termExtractor);
+    for (String term : termExtractor.terms) {
+      semanticQueryText.append(term).append(' ');
+    }
+    if (semanticQueryText.length() > 0) {

Review comment:
       I added a test case  - you get zero




-- 
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 merged pull request #241: LUCENE-10016: Added KnnVector index/query support to demo

Posted by GitBox <gi...@apache.org>.
msokolov merged pull request #241:
URL: https://github.com/apache/lucene/pull/241


   


-- 
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] jtibshirani commented on a change in pull request #241: LUCENE-10016: Added KnnVector index/query support to demo

Posted by GitBox <gi...@apache.org>.
jtibshirani commented on a change in pull request #241:
URL: https://github.com/apache/lucene/pull/241#discussion_r689652805



##########
File path: lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java
##########
@@ -242,4 +266,54 @@ public static void doPagingSearch(
       }
     }
   }
+
+  private static Query addSemanticQuery(Query query, KnnVectorDict vectorDict) throws IOException {
+    StringBuilder semanticQueryText = new StringBuilder();
+    QueryFieldTermExtractor termExtractor = new QueryFieldTermExtractor("contents");
+    query.visit(termExtractor);
+    for (String term : termExtractor.terms) {
+      semanticQueryText.append(term).append(' ');

Review comment:
       I found it surprising that this demo incorporates an analyzer -- I was expecting users to map directly from the query string or query terms to a vector. I'm guessing many users will be using sentence embedding models, which take a whole (untokenized) sentence as input and output a vector.
   
   In this demo, would it make sense to simplify and have a method like "createEmbeddingVector" that maps directly from a set of terms to a vector?

##########
File path: lucene/demo/src/java/org/apache/lucene/demo/SearchFiles.java
##########
@@ -242,4 +266,54 @@ public static void doPagingSearch(
       }
     }
   }
+
+  private static Query addSemanticQuery(Query query, KnnVectorDict vectorDict) throws IOException {
+    StringBuilder semanticQueryText = new StringBuilder();
+    QueryFieldTermExtractor termExtractor = new QueryFieldTermExtractor("contents");
+    query.visit(termExtractor);
+    for (String term : termExtractor.terms) {
+      semanticQueryText.append(term).append(' ');
+    }
+    if (semanticQueryText.length() > 0) {
+      KnnVectorQuery knnQuery =
+          new KnnVectorQuery(
+              "contents-vector",
+              new DemoKnnAnalyzer(vectorDict).analyze("text", semanticQueryText.toString()),
+              1);

Review comment:
       It might make sense to use a k greater than 1 here, to make it clear to users that they can retrieve any number of nearest vectors. Maybe we could even make it configurable instead of just having a flag to turn vectors on/ off.




-- 
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 #241: LUCENE-10016: Added KnnVector index/query support to demo

Posted by GitBox <gi...@apache.org>.
msokolov commented on pull request #241:
URL: https://github.com/apache/lucene/pull/241#issuecomment-898904915


   I fixed a few small issues, and plan to merge after the weekend.


-- 
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 change in pull request #241: LUCENE-10016: Added KnnVector index/query support to demo

Posted by GitBox <gi...@apache.org>.
msokolov commented on a change in pull request #241:
URL: https://github.com/apache/lucene/pull/241#discussion_r689807397



##########
File path: lucene/demo/src/java/org/apache/lucene/demo/IndexFiles.java
##########
@@ -201,6 +235,18 @@ static void indexDoc(IndexWriter writer, Path file, long lastModified) throws IO
               "contents",
               new BufferedReader(new InputStreamReader(stream, StandardCharsets.UTF_8))));
 
+      if (dict != null) {

Review comment:
       fair, and for didactic purposes we really especially want clarity here - I'll change to `vectorDict`. 




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