Re: [PR] Early terminate visit BKD leaf when current value greater than upper point in sorted dim. [lucene]

2024-05-11 Thread via GitHub


vsop-479 commented on PR #12528:
URL: https://github.com/apache/lucene/pull/12528#issuecomment-2105633049

   @iverase 
   I implemented `visitWithSortedDim` for `pointCount` in `PointRangeQuery`. 
Please take a look when you get a chance.
   
   Or Do you have any other ideas to terminate visiting points, which already 
greater than `upperPoint`? Even just for 1D points.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

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


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



Re: [PR] Use `IndexInput#prefetch` for terms dictionary lookups. [lucene]

2024-05-11 Thread via GitHub


jpountz commented on PR #13359:
URL: https://github.com/apache/lucene/pull/13359#issuecomment-2105658311

   This is a draft as I need to do more work on tests and making sure that this 
new method cannot corrupt the state of the `SegmentTermsEnum`.
   
   
   But I created a benchmark that starts looking like running a Lucene 
query that is encouraging. It creates an index with many terms that have very 
short postings lists, so that running boolean queries on these terms is heavy 
on terms dictionary lookups rather than reading postings. Then it manually runs 
a disjunction over 3 terms (some of these terms may not exist in the index as 
they are created randomly), computing how long it takes to evaluate all hits. 
To work properly when running a query, we'd need to move `#bulkScorer` from 
`Weight` to `ScorerSupplier`, which I intend to do as a follow-up.
   
   ```java
   import java.io.IOException;
   import java.io.UncheckedIOException;
   import java.nio.file.Path;
   import java.nio.file.Paths;
   import java.util.ArrayList;
   import java.util.List;
   import java.util.Random;
   import java.util.concurrent.ExecutorService;
   import java.util.concurrent.Executors;
   import java.util.concurrent.ThreadLocalRandom;
   import java.util.concurrent.TimeUnit;
   import java.util.concurrent.atomic.AtomicLong;
   
   import org.apache.lucene.document.Document;
   import org.apache.lucene.document.Field.Store;
   import org.apache.lucene.document.StringField;
   import org.apache.lucene.index.DirectoryReader;
   import org.apache.lucene.index.IndexReader;
   import org.apache.lucene.index.IndexWriter;
   import org.apache.lucene.index.IndexWriterConfig;
   import org.apache.lucene.index.Term;
   import org.apache.lucene.index.TieredMergePolicy;
   import org.apache.lucene.search.BooleanClause.Occur;
   import org.apache.lucene.search.BooleanQuery;
   import org.apache.lucene.search.DocIdSetIterator;
   import org.apache.lucene.search.IndexSearcher;
   import org.apache.lucene.search.ScoreMode;
   import org.apache.lucene.search.Scorer;
   import org.apache.lucene.search.ScorerSupplier;
   import org.apache.lucene.search.TermQuery;
   import org.apache.lucene.search.Weight;
   import org.apache.lucene.store.Directory;
   import org.apache.lucene.store.FSDirectory;
   
   public class TermsEnumPrefetchBench {
   
 private static final int NUM_TERMS = 3;
 public static int DUMMY;
 
 public static void main(String[] args) throws Exception {
   Path dirPath = Paths.get(args[0]);
   Directory dir = FSDirectory.open(dirPath);
   if (DirectoryReader.indexExists(dir) == false) {
 TieredMergePolicy mp = new TieredMergePolicy();
 mp.setSegmentsPerTier(100);
 mp.setMaxMergeAtOnce(100);
 mp.setMaxMergedSegmentMB(1024);
 try (IndexWriter w = new IndexWriter(dir, new IndexWriterConfig()
 .setMergePolicy(mp)
 .setRAMBufferSizeMB(1024))) {
   ExecutorService executor = 
Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors());
   AtomicLong indexed = new AtomicLong(0);
   for (int task = 0; task < 1000; ++task) {
 executor.execute(() -> {
   Random r = ThreadLocalRandom.current();
   for (int i = 0; i < 1_000; ++i) {
 Document doc = new Document();
 for (int j = 0; j < 10_000; ++j) {
   doc.add(new StringField("f", 
Long.toString(r.nextLong(20_000_000_000L)), Store.NO));
 }
 try {
   w.addDocument(doc);
 } catch (IOException e) {
   throw new UncheckedIOException(e);
 }
 final long actualIndexed = indexed.incrementAndGet(); 
 if (actualIndexed % 10_000 == 0) {
   System.out.println("Indexed: " + actualIndexed);
 }
   }
 });
   }
   
   executor.shutdown();
   executor.awaitTermination(1, TimeUnit.DAYS);
   w.commit();
   System.out.println("Start force merging");
   w.forceMerge(1);
   System.out.println("Done force merging");
   w.commit();
 }
   }
   List latencies = new ArrayList<>();
   try (IndexReader reader = DirectoryReader.open(dir)) {
 IndexSearcher searcher = new IndexSearcher(reader);
   
 Random r = ThreadLocalRandom.current();
 for (int i = 0; i < 10_000; ++i) {
   long start = System.nanoTime();
   BooleanQuery.Builder query = new BooleanQuery.Builder();
   for (int t = 0; t < NUM_TERMS; ++t) {
 query.add(new TermQuery(new Term("f", 
Long.toString(r.nextLong(20_000_000_000L, Occur.SHOULD);
   }
   Weight weight = 
searcher.createWeight(searcher.rewrite(query.build()), 
ScoreMode.COMPLETE_NO_SCORES, 1f);
   ScorerSupplier ss = weight.scorerSuppl

Re: [I] Multi-value Support for KnnVectorField [lucene]

2024-05-11 Thread via GitHub


krickert commented on issue #12313:
URL: https://github.com/apache/lucene/issues/12313#issuecomment-2105745361

   I was thinking about this and thought this would be cool with a few 
different use cases for a multi-valued vector:
   
   1. The multi-values are treated the same as the single value, except once 
it's found to be a nearest K, it won't repeat.  For example: Doc A has vectors 
A1, A2, and A3.  Doc2 has vectors B1 bad B2.  Then we have a Doc3 with C1. A 
vector search is performed, and the K'th nearest return:
   A1
   A2
   C1
   B2
   B1
   A3
   
   In one scenerio, the search results would be the same as above, and the docs 
would repeat.  
   
   In another scenario, the results would just return the top doc and not 
repeat it.  So a KNN result would be:
   Doc1 (A1 won)
   Doc3 (C1 won)
   Doc2 (B2 won)
   
   ... 
   
   In another option, we can look into indexing the vectors where we get an 
average, min, or max between each dimension and just index the avg, min, or 
max.  For some reason, I think this might be a bit weird since you can do these 
calculations at index time.  But just a thought...
   
   Are any of the suggestions similar to what I'm suggesting?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

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


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



Re: [PR] Reduce memory usage of field maps in FieldInfos and BlockTree TermsReader. [lucene]

2024-05-11 Thread via GitHub


bruno-roustant commented on code in PR #13327:
URL: https://github.com/apache/lucene/pull/13327#discussion_r1597449679


##
lucene/core/src/java/org/apache/lucene/index/FieldInfos.java:
##
@@ -139,17 +148,38 @@ public FieldInfos(FieldInfo[] infos) {
 this.softDeletesField = softDeletesField;
 this.parentField = parentField;
 
-FieldInfo[] sortedFieldInfos = ArrayUtil.copyOfSubArray(infos, 0, 
infos.length);
-Arrays.sort(sortedFieldInfos, (fi1, fi2) -> Integer.compare(fi1.number, 
fi2.number));
-int maxFieldNumber = infos.length == 0 ? -1 : 
sortedFieldInfos[infos.length - 1].number;
-// If there are many fields and the max field number is greater than twice 
the number
-// of fields, then a map structure is more compact to store the by-number 
mapping.
-byNumber =
-maxFieldNumber >= 2 * infos.length && maxFieldNumber >= 32
-? new MapFieldInfoByNumber(infos)
-: new ArrayFieldInfoByNumber(infos, maxFieldNumber);
-// The iteration of FieldInfo is ordered by ascending field number.
-values = 
Collections.unmodifiableCollection(Arrays.asList(sortedFieldInfos));
+if (fieldNumberStrictlyAscending && maxFieldNumber == infos.length - 1) {
+  // The input FieldInfo[] contains all fields numbered from 0 to 
infos.length - 1 and they are
+  // sorted, use it directly. This is an optimization when reading a 
segment with all fields
+  // since the FieldInfo[] is sorted.
+  byNumber = infos; // We could copy the input array, but do we need to?
+  values = Arrays.asList(byNumber);
+} else {
+  byNumber = new FieldInfo[maxFieldNumber + 1];
+  for (FieldInfo fieldInfo : infos) {
+FieldInfo previous = byNumber[fieldInfo.number];

Review Comment:
   Yes, existing, to check we have no duplicates. Let's rename it "existing" 
for clarity.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

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


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



Re: [PR] Reduce memory usage of field maps in FieldInfos and BlockTree TermsReader. [lucene]

2024-05-11 Thread via GitHub


dsmiley commented on code in PR #13327:
URL: https://github.com/apache/lucene/pull/13327#discussion_r1597469133


##
lucene/core/src/java/org/apache/lucene/index/FieldInfos.java:
##
@@ -139,17 +148,38 @@ public FieldInfos(FieldInfo[] infos) {
 this.softDeletesField = softDeletesField;
 this.parentField = parentField;
 
-FieldInfo[] sortedFieldInfos = ArrayUtil.copyOfSubArray(infos, 0, 
infos.length);
-Arrays.sort(sortedFieldInfos, (fi1, fi2) -> Integer.compare(fi1.number, 
fi2.number));
-int maxFieldNumber = infos.length == 0 ? -1 : 
sortedFieldInfos[infos.length - 1].number;
-// If there are many fields and the max field number is greater than twice 
the number
-// of fields, then a map structure is more compact to store the by-number 
mapping.
-byNumber =
-maxFieldNumber >= 2 * infos.length && maxFieldNumber >= 32
-? new MapFieldInfoByNumber(infos)
-: new ArrayFieldInfoByNumber(infos, maxFieldNumber);
-// The iteration of FieldInfo is ordered by ascending field number.
-values = 
Collections.unmodifiableCollection(Arrays.asList(sortedFieldInfos));
+if (fieldNumberStrictlyAscending && maxFieldNumber == infos.length - 1) {
+  // The input FieldInfo[] contains all fields numbered from 0 to 
infos.length - 1 and they are
+  // sorted, use it directly. This is an optimization when reading a 
segment with all fields
+  // since the FieldInfo[] is sorted.
+  byNumber = infos; // We could copy the input array, but do we need to?
+  values = Arrays.asList(byNumber);
+} else {
+  byNumber = new FieldInfo[maxFieldNumber + 1];
+  for (FieldInfo fieldInfo : infos) {
+FieldInfo previous = byNumber[fieldInfo.number];

Review Comment:
   Thanks.  "previous" wasn't wrong, just ambiguous.  I like "existing"; 
another possible name is "old".



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

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


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



[PR] Call ArrayUtil.copyOf instead of ArrayUtil.copySubArray for full array copy. [lucene]

2024-05-11 Thread via GitHub


bruno-roustant opened a new pull request, #13360:
URL: https://github.com/apache/lucene/pull/13360

   ArrayUtil provides copySubArray() as replacement of Array.copyOf and 
Array.copyOfRange (with better bounds check). Unfortunately ArrayUtil does not 
provide copyArray(), so it is weird for callers to have to call a method named 
copySubArray() when we want to copy the full array.
   
   This PR adds ArrayUtil.copyArray(), and replaces the calls to 
ArrayUtil.copySubArray(array, 0, array.length) with calls to 
ArrayUtil.copyArray(array).
   I used the following regex to match all calls:
   `copyOfSubArray\(([^,]+), 0, .+\.length\)
   `and this one to replace
   `copyOf($1)`


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

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


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



Re: [PR] Reduce memory usage of field maps in FieldInfos and BlockTree TermsReader. [lucene]

2024-05-11 Thread via GitHub


bruno-roustant commented on code in PR #13327:
URL: https://github.com/apache/lucene/pull/13327#discussion_r1597476337


##
lucene/core/src/java/org/apache/lucene/index/FieldInfos.java:
##
@@ -139,17 +148,38 @@ public FieldInfos(FieldInfo[] infos) {
 this.softDeletesField = softDeletesField;
 this.parentField = parentField;
 
-FieldInfo[] sortedFieldInfos = ArrayUtil.copyOfSubArray(infos, 0, 
infos.length);
-Arrays.sort(sortedFieldInfos, (fi1, fi2) -> Integer.compare(fi1.number, 
fi2.number));
-int maxFieldNumber = infos.length == 0 ? -1 : 
sortedFieldInfos[infos.length - 1].number;
-// If there are many fields and the max field number is greater than twice 
the number
-// of fields, then a map structure is more compact to store the by-number 
mapping.
-byNumber =
-maxFieldNumber >= 2 * infos.length && maxFieldNumber >= 32
-? new MapFieldInfoByNumber(infos)
-: new ArrayFieldInfoByNumber(infos, maxFieldNumber);
-// The iteration of FieldInfo is ordered by ascending field number.
-values = 
Collections.unmodifiableCollection(Arrays.asList(sortedFieldInfos));
+if (fieldNumberStrictlyAscending && maxFieldNumber == infos.length - 1) {
+  // The input FieldInfo[] contains all fields numbered from 0 to 
infos.length - 1 and they are
+  // sorted, use it directly. This is an optimization when reading a 
segment with all fields
+  // since the FieldInfo[] is sorted.
+  byNumber = infos; // We could copy the input array, but do we need to?
+  values = Arrays.asList(byNumber);
+} else {
+  byNumber = new FieldInfo[maxFieldNumber + 1];
+  for (FieldInfo fieldInfo : infos) {
+FieldInfo previous = byNumber[fieldInfo.number];
+if (previous != null) {
+  throw new IllegalArgumentException(
+  "duplicate field numbers: "
+  + previous.name
+  + " and "
+  + fieldInfo.name
+  + " have: "
+  + fieldInfo.number);
+}
+byNumber[fieldInfo.number] = fieldInfo;
+  }
+  if (maxFieldNumber == infos.length - 1) {
+// No fields are missing, use byNumber.
+values = Arrays.asList(byNumber);
+  } else {
+// The below code is faster than 
Arrays.stream(byNumber).filter(Objects::nonNull).toList(),
+// mainly when the input FieldInfo[] is sorted, when reading a segment.
+FieldInfo[] sortedFieldInfos = ArrayUtil.copyOfSubArray(infos, 0, 
infos.length);

Review Comment:
   I agree. I created another [PR](https://github.com/apache/lucene/pull/13360) 
to add ArrayUtil.copyOf().



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

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


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



Re: [PR] Reduce memory usage of field maps in FieldInfos and BlockTree TermsReader. [lucene]

2024-05-11 Thread via GitHub


bruno-roustant commented on code in PR #13327:
URL: https://github.com/apache/lucene/pull/13327#discussion_r1585022423


##
lucene/core/src/java/org/apache/lucene/index/FieldInfos.java:
##
@@ -156,15 +139,17 @@ public FieldInfos(FieldInfo[] infos) {
 this.softDeletesField = softDeletesField;
 this.parentField = parentField;
 
-List valuesTemp = new ArrayList<>(infos.length);
-byNumber = new FieldInfo[size];
-for (int i = 0; i < size; i++) {
-  byNumber[i] = byNumberTemp[i];
-  if (byNumberTemp[i] != null) {
-valuesTemp.add(byNumberTemp[i]);
-  }
-}
-values = Collections.unmodifiableCollection(valuesTemp);
+FieldInfo[] sortedFieldInfos = ArrayUtil.copyOfSubArray(infos, 0, 
infos.length);
+Arrays.sort(sortedFieldInfos, (fi1, fi2) -> Integer.compare(fi1.number, 
fi2.number));
+int maxFieldNumber = infos.length == 0 ? -1 : 
sortedFieldInfos[infos.length - 1].number;
+// If there are many fields and the max field number is greater than twice 
the number
+// of fields, then a map structure is more compact to store the by-number 
mapping.
+byNumber =
+maxFieldNumber >= 2 * infos.length && maxFieldNumber >= 32
+? new MapFieldInfoByNumber(infos)
+: new ArrayFieldInfoByNumber(infos, maxFieldNumber);
+// The iteration of FieldInfo is ordered by ascending field number.
+values = 
Collections.unmodifiableCollection(Arrays.asList(sortedFieldInfos));

Review Comment:
   List.of() makes a copy as it considers the input as an "untrusted array". 
Here we don't copy, just wrap. Actually we could keep just 
Arrays.asList(sortedFieldInfos) since we own it privately, so we know we don't 
modify it (only iterator(), which does not support removal for Arrays.asList).



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

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


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



Re: [PR] Reduce memory usage of field maps in FieldInfos and BlockTree TermsReader. [lucene]

2024-05-11 Thread via GitHub


dsmiley commented on code in PR #13327:
URL: https://github.com/apache/lucene/pull/13327#discussion_r1597484741


##
lucene/core/src/java/org/apache/lucene/index/FieldInfos.java:
##
@@ -139,17 +148,38 @@ public FieldInfos(FieldInfo[] infos) {
 this.softDeletesField = softDeletesField;
 this.parentField = parentField;
 
-FieldInfo[] sortedFieldInfos = ArrayUtil.copyOfSubArray(infos, 0, 
infos.length);
-Arrays.sort(sortedFieldInfos, (fi1, fi2) -> Integer.compare(fi1.number, 
fi2.number));
-int maxFieldNumber = infos.length == 0 ? -1 : 
sortedFieldInfos[infos.length - 1].number;
-// If there are many fields and the max field number is greater than twice 
the number
-// of fields, then a map structure is more compact to store the by-number 
mapping.
-byNumber =
-maxFieldNumber >= 2 * infos.length && maxFieldNumber >= 32
-? new MapFieldInfoByNumber(infos)
-: new ArrayFieldInfoByNumber(infos, maxFieldNumber);
-// The iteration of FieldInfo is ordered by ascending field number.
-values = 
Collections.unmodifiableCollection(Arrays.asList(sortedFieldInfos));
+if (fieldNumberStrictlyAscending && maxFieldNumber == infos.length - 1) {
+  // The input FieldInfo[] contains all fields numbered from 0 to 
infos.length - 1 and they are
+  // sorted, use it directly. This is an optimization when reading a 
segment with all fields
+  // since the FieldInfo[] is sorted.
+  byNumber = infos; // We could copy the input array, but do we need to?
+  values = Arrays.asList(byNumber);
+} else {
+  byNumber = new FieldInfo[maxFieldNumber + 1];
+  for (FieldInfo fieldInfo : infos) {
+FieldInfo previous = byNumber[fieldInfo.number];
+if (previous != null) {
+  throw new IllegalArgumentException(
+  "duplicate field numbers: "
+  + previous.name
+  + " and "
+  + fieldInfo.name
+  + " have: "
+  + fieldInfo.number);
+}
+byNumber[fieldInfo.number] = fieldInfo;
+  }
+  if (maxFieldNumber == infos.length - 1) {
+// No fields are missing, use byNumber.
+values = Arrays.asList(byNumber);
+  } else {
+// The below code is faster than 
Arrays.stream(byNumber).filter(Objects::nonNull).toList(),
+// mainly when the input FieldInfo[] is sorted, when reading a segment.
+FieldInfo[] sortedFieldInfos = ArrayUtil.copyOfSubArray(infos, 0, 
infos.length);

Review Comment:
   Don't even need to copy it as we've given ourselves permission to take 
ownership of the input array; to manipulate it.



##
lucene/core/src/java/org/apache/lucene/index/FieldInfos.java:
##
@@ -156,15 +151,38 @@ public FieldInfos(FieldInfo[] infos) {
 this.softDeletesField = softDeletesField;
 this.parentField = parentField;
 
-List valuesTemp = new ArrayList<>(infos.length);
-byNumber = new FieldInfo[size];
-for (int i = 0; i < size; i++) {
-  byNumber[i] = byNumberTemp[i];
-  if (byNumberTemp[i] != null) {
-valuesTemp.add(byNumberTemp[i]);
+if (fieldNumberStrictlyAscending && maxFieldNumber == infos.length - 1) {
+  // The input FieldInfo[] contains all fields numbered from 0 to 
infos.length - 1, and they are
+  // sorted, use it directly. This is an optimization when reading a 
segment with all fields
+  // since the FieldInfo[] is sorted.
+  byNumber = infos;
+  values = Arrays.asList(byNumber);
+} else {
+  byNumber = new FieldInfo[maxFieldNumber + 1];
+  for (FieldInfo fieldInfo : infos) {
+FieldInfo existing = byNumber[fieldInfo.number];
+if (existing != null) {
+  throw new IllegalArgumentException(
+  "duplicate field numbers: "
+  + existing.name
+  + " and "
+  + fieldInfo.name
+  + " have: "
+  + fieldInfo.number);
+}
+byNumber[fieldInfo.number] = fieldInfo;
+  }
+  if (maxFieldNumber == infos.length - 1) {
+// No fields are missing, use byNumber.
+values = Arrays.asList(byNumber);
+  } else {
+// The below code is faster than 
Arrays.stream(byNumber).filter(Objects::nonNull).toList(),
+// mainly when the input FieldInfo[] is sorted, when reading a segment.

Review Comment:
   If fieldNumberStrictlyAscending, and we've given ourselves permission to 
accept the input array, we can merely wrap it in Arrays.asList and we're 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-unsubscr...@lucene.apache.org

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


-

Re: [PR] Reduce memory usage of field maps in FieldInfos and BlockTree TermsReader. [lucene]

2024-05-11 Thread via GitHub


bruno-roustant commented on code in PR #13327:
URL: https://github.com/apache/lucene/pull/13327#discussion_r1597498694


##
lucene/core/src/java/org/apache/lucene/index/FieldInfos.java:
##
@@ -156,15 +151,38 @@ public FieldInfos(FieldInfo[] infos) {
 this.softDeletesField = softDeletesField;
 this.parentField = parentField;
 
-List valuesTemp = new ArrayList<>(infos.length);
-byNumber = new FieldInfo[size];
-for (int i = 0; i < size; i++) {
-  byNumber[i] = byNumberTemp[i];
-  if (byNumberTemp[i] != null) {
-valuesTemp.add(byNumberTemp[i]);
+if (fieldNumberStrictlyAscending && maxFieldNumber == infos.length - 1) {
+  // The input FieldInfo[] contains all fields numbered from 0 to 
infos.length - 1, and they are
+  // sorted, use it directly. This is an optimization when reading a 
segment with all fields
+  // since the FieldInfo[] is sorted.
+  byNumber = infos;
+  values = Arrays.asList(byNumber);
+} else {
+  byNumber = new FieldInfo[maxFieldNumber + 1];
+  for (FieldInfo fieldInfo : infos) {
+FieldInfo existing = byNumber[fieldInfo.number];
+if (existing != null) {
+  throw new IllegalArgumentException(
+  "duplicate field numbers: "
+  + existing.name
+  + " and "
+  + fieldInfo.name
+  + " have: "
+  + fieldInfo.number);
+}
+byNumber[fieldInfo.number] = fieldInfo;
+  }
+  if (maxFieldNumber == infos.length - 1) {
+// No fields are missing, use byNumber.
+values = Arrays.asList(byNumber);
+  } else {
+// The below code is faster than 
Arrays.stream(byNumber).filter(Objects::nonNull).toList(),
+// mainly when the input FieldInfo[] is sorted, when reading a segment.

Review Comment:
   Here maxFieldNumber > infos.length - 1, so some fields are missing in 
byNumber (byNumber[fieldNumber] == null). We must not return those null 
FieldInfos in values.iterator().



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

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


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



Re: [PR] Reduce memory usage of field maps in FieldInfos and BlockTree TermsReader. [lucene]

2024-05-11 Thread via GitHub


bruno-roustant commented on code in PR #13327:
URL: https://github.com/apache/lucene/pull/13327#discussion_r1597498694


##
lucene/core/src/java/org/apache/lucene/index/FieldInfos.java:
##
@@ -156,15 +151,38 @@ public FieldInfos(FieldInfo[] infos) {
 this.softDeletesField = softDeletesField;
 this.parentField = parentField;
 
-List valuesTemp = new ArrayList<>(infos.length);
-byNumber = new FieldInfo[size];
-for (int i = 0; i < size; i++) {
-  byNumber[i] = byNumberTemp[i];
-  if (byNumberTemp[i] != null) {
-valuesTemp.add(byNumberTemp[i]);
+if (fieldNumberStrictlyAscending && maxFieldNumber == infos.length - 1) {
+  // The input FieldInfo[] contains all fields numbered from 0 to 
infos.length - 1, and they are
+  // sorted, use it directly. This is an optimization when reading a 
segment with all fields
+  // since the FieldInfo[] is sorted.
+  byNumber = infos;
+  values = Arrays.asList(byNumber);
+} else {
+  byNumber = new FieldInfo[maxFieldNumber + 1];
+  for (FieldInfo fieldInfo : infos) {
+FieldInfo existing = byNumber[fieldInfo.number];
+if (existing != null) {
+  throw new IllegalArgumentException(
+  "duplicate field numbers: "
+  + existing.name
+  + " and "
+  + fieldInfo.name
+  + " have: "
+  + fieldInfo.number);
+}
+byNumber[fieldInfo.number] = fieldInfo;
+  }
+  if (maxFieldNumber == infos.length - 1) {
+// No fields are missing, use byNumber.
+values = Arrays.asList(byNumber);
+  } else {
+// The below code is faster than 
Arrays.stream(byNumber).filter(Objects::nonNull).toList(),
+// mainly when the input FieldInfo[] is sorted, when reading a segment.

Review Comment:
   Here maxFieldNumber > infos.length - 1, so some fields are missing in 
byNumber (byNumber[fieldNumber] == null). We must not return those null 
FieldInfos in values.iterator().



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

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


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



Re: [PR] Use `IndexInput#prefetch` for terms dictionary lookups. [lucene]

2024-05-11 Thread via GitHub


rmuir commented on code in PR #13359:
URL: https://github.com/apache/lucene/pull/13359#discussion_r1597522761


##
lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnum.java:
##
@@ -307,6 +309,31 @@ private boolean setEOF() {
 return true;
   }
 
+  @Override
+  public void prepareSeekExact(BytesRef target) throws IOException {
+if (fr.index == null) {
+  throw new IllegalStateException("terms index was not loaded");
+}
+
+if (fr.size() == 0 || target.compareTo(fr.getMin()) < 0 || 
target.compareTo(fr.getMax()) > 0) {
+  return;
+}
+
+// TODO: should we try to reuse the current state of this terms enum when 
applicable?
+BytesRefFSTEnum indexEnum = new BytesRefFSTEnum<>(fr.index);
+InputOutput output = indexEnum.seekFloor(target);
+final long code =
+fr.readVLongOutput(
+new ByteArrayDataInput(
+output.output.bytes, output.output.offset, 
output.output.length));
+final long fpSeek = code >>> 
Lucene90BlockTreeTermsReader.OUTPUT_FLAGS_NUM_BITS;
+initIndexInput();
+final long fp = in.getFilePointer();
+in.seek(fpSeek);
+in.prefetch(1); // TODO: could we know the length of the block?
+in.seek(fp); // TODO: do we actually need to do this?

Review Comment:
   I really don't like these calls to `seek()` just to prefetch data. Since it 
is just prefetching, I'd prefer if this "dance" was an impl detail, if needed. 
   It would make the code simpler to just pass parameter to prefetch rather 
than do this.
   
   Then it is clear that the default implementation won't cause harm 
(unnecessary io) for any directory subclasses 
   
   So I think prefetch should take location as argument? It is just a hint and 
not real i/o by the thread. It's intentionally not sequential and sequential 
API for it only hurts.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@lucene.apache.org

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


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