Re: [PR] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579103250 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: can we replace `LinkedHashMap ` with `ConcurrentHashMap` ? -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579103250 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: can we adjust `LinkedHashMap ` to be `ConcurrentHashMap` ? -- 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] Improve int4 compressed comparisons performance [lucene]
ChrisHegarty commented on code in PR #13321: URL: https://github.com/apache/lucene/pull/13321#discussion_r1579117412 ## lucene/core/src/java/org/apache/lucene/util/VectorUtil.java: ## @@ -179,7 +179,30 @@ public static int int4DotProduct(byte[] a, byte[] b) { if (a.length != b.length) { throw new IllegalArgumentException("vector dimensions differ: " + a.length + "!=" + b.length); } -return IMPL.int4DotProduct(a, b); +return IMPL.int4DotProduct(a, false, b, false); + } + + /** + * Dot product computed over int4 (values between [0,15]) bytes. The second vector is considered + * "packed" (i.e. every two bytes represent a single value). The following packing is assumed: Review Comment: is this not "every byte represents two values" ? ## lucene/core/src/java/org/apache/lucene/codecs/lucene99/Lucene99ScalarQuantizedVectorScorer.java: ## @@ -0,0 +1,283 @@ +/* + * 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.codecs.lucene99; + +import static org.apache.lucene.codecs.hnsw.ScalarQuantizedVectorScorer.quantizeQuery; +import static org.apache.lucene.util.VectorUtil.scaleMaxInnerProductScore; + +import java.io.IOException; +import org.apache.lucene.codecs.hnsw.FlatVectorsScorer; +import org.apache.lucene.index.VectorSimilarityFunction; +import org.apache.lucene.util.VectorUtil; +import org.apache.lucene.util.hnsw.RandomAccessVectorValues; +import org.apache.lucene.util.hnsw.RandomVectorScorer; +import org.apache.lucene.util.hnsw.RandomVectorScorerSupplier; +import org.apache.lucene.util.quantization.RandomAccessQuantizedByteVectorValues; +import org.apache.lucene.util.quantization.ScalarQuantizer; + +/** + * Optimized scalar quantized implementation of {@link FlatVectorsScorer} for quantized vectors + * stored in the Lucene99 format. + * + * @lucene.experimental + */ +public class Lucene99ScalarQuantizedVectorScorer implements FlatVectorsScorer { + + private final FlatVectorsScorer nonQuantizedDelegate; + + public Lucene99ScalarQuantizedVectorScorer(FlatVectorsScorer flatVectorsScorer) { +nonQuantizedDelegate = flatVectorsScorer; + } + + @Override + public RandomVectorScorerSupplier getRandomVectorScorerSupplier( + VectorSimilarityFunction similarityFunction, RandomAccessVectorValues vectorValues) + throws IOException { +if (vectorValues instanceof RandomAccessQuantizedByteVectorValues quantizedByteVectorValues) { + return new ScalarQuantizedRandomVectorScorerSupplier( + quantizedByteVectorValues, similarityFunction); +} +// It is possible to get to this branch during initial indexing and flush +return nonQuantizedDelegate.getRandomVectorScorerSupplier(similarityFunction, vectorValues); + } + + @Override + public RandomVectorScorer getRandomVectorScorer( + VectorSimilarityFunction similarityFunction, + RandomAccessVectorValues vectorValues, + float[] target) + throws IOException { +if (vectorValues instanceof RandomAccessQuantizedByteVectorValues quantizedByteVectorValues) { + ScalarQuantizer scalarQuantizer = quantizedByteVectorValues.getScalarQuantizer(); + byte[] targetBytes = new byte[target.length]; + float offsetCorrection = + quantizeQuery(target, targetBytes, similarityFunction, scalarQuantizer); + return fromVectorSimilarity( + targetBytes, + offsetCorrection, + similarityFunction, + scalarQuantizer.getConstantMultiplier(), + quantizedByteVectorValues); +} +// It is possible to get to this branch during initial indexing and flush +return nonQuantizedDelegate.getRandomVectorScorer(similarityFunction, vectorValues, target); + } + + @Override + public RandomVectorScorer getRandomVectorScorer( + VectorSimilarityFunction similarityFunction, + RandomAccessVectorValues vectorValues, + byte[] target) + throws IOException { +return nonQuantizedDelegate.getRandomVectorScorer(similarityFunction, vectorValues, target); + } + + @Override + public String toString() { +return "ScalarQuantizedVectorScorer(" + "nonQuantizedDelegate=" + nonQuantizedDelegate + ')'; +
Re: [PR] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579173947 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: I adjust `cache` AND `uniqueQueries` to be `ConcurrentHashMap` objects and get the benchmark results as follows. doc count | field cardinality | query point | baseline QPS | candidate QPS | diff percentage -- | -- | -- | -- | -- | -- 3000 | 10 | 1 | 2481 | 5102 | 105.6% 3000 | 10 | 1 | 2481 | 4843 | 95.2% (using `LongAdder `) 3000 | 10 | 1 | 2481 | 3851 | 55% (using `LongAdder ` And `ConcurrentHashMap`) -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579173947 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: I adjust `cache` AND `uniqueQueries` to be `ConcurrentHashMap` objects and get the benchmark results as follows. doc count | field cardinality | query point | baseline QPS | candidate QPS | diff percentage -- | -- | -- | -- | -- | -- 3000 | 10 | 1 | 2481 | 5102 | 105.6% 3000 | 10 | 1 | 2481 | 4843 | 95.2% (using `LongAdder `) 3000 | 10 | 1 | 2481 | 3851 | 55% (using `LongAdder ` And `ConcurrentHashMap`) -- 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: [I] Query parser fails to parse a range query string when there are escaped brackets inside the range [lucene]
marko-bekhta commented on issue #13234: URL: https://github.com/apache/lucene/issues/13234#issuecomment-2076897782 Thanks for looking at this and for the suggestion! I've also tested it out and can confirm that it worked. I'll let you decide how you'd want to proceed with this ticket (looking at the linked PR, you are considering whether an update to the parser should be applied to support more query string variations) -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579285322 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: In ConcurrentHashMap, although the order of insertion operations is uncertain, it ensures the thread safety of insertion operations. Multiple threads can perform insert operations simultaneously without causing data inconsistency or corruption. -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579285322 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: In `ConcurrentHashMap`, although the order of insertion operations is uncertain, it ensures the thread safety of insertion operations. Multiple threads can perform insert operations simultaneously without causing data inconsistency or corruption. -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579285322 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: In `ConcurrentHashMap`, although the order of insertion operations is uncertain, it ensures the thread safety of insertion operations. Multiple threads can perform insert operations simultaneously without causing data inconsistency or corruption. Still a 55% improvement. -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579285322 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: In `ConcurrentHashMap`, although the order of insertion operations is uncertain, which leads to test failure, it ensures the thread safety of insertion operations. Multiple threads can perform insert operations simultaneously without causing data inconsistency or corruption. It still has a 55% improvement. -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579285322 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: In `ConcurrentHashMap`, although the order of insertion operations is uncertain, which leads to test failure, it ensures the thread safety of insertion operations and still has a 55% improvement. -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
benwtrent commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579341971 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: How do you suggest that we achieve the "LRU" aspect of this cache without knowing the queries that were used most recently? The point of the linked hashmap is that it orders by last access. We need something like this to achieve LRU expiration -- 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] Improve int4 compressed comparisons performance [lucene]
benwtrent commented on code in PR #13321: URL: https://github.com/apache/lucene/pull/13321#discussion_r1579422728 ## lucene/core/src/java/org/apache/lucene/util/VectorUtil.java: ## @@ -179,7 +179,30 @@ public static int int4DotProduct(byte[] a, byte[] b) { if (a.length != b.length) { throw new IllegalArgumentException("vector dimensions differ: " + a.length + "!=" + b.length); } -return IMPL.int4DotProduct(a, b); +return IMPL.int4DotProduct(a, false, b, false); + } + + /** + * Dot product computed over int4 (values between [0,15]) bytes. The second vector is considered + * "packed" (i.e. every two bytes represent a single value). The following packing is assumed: Review Comment: Yep. -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579458103 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: I understand your point. I try to use `Collections.synchronizedMap` and get the benchmark results as follows doc count | field cardinality | query point | baseline QPS | candidate QPS | diff percentage -- | -- | -- | -- | -- | -- 3000 | 10 | 1 | 2481 | 5102 | 105.6% 3000 | 10 | 1 | 2481 | 4843 | 95.2% (using `LongAdder `) 3000 | 10 | 1 | 2481 | 3751 | 51% (using `LongAdder ` And `Collections.synchronizedMap`) It seems that performance can be improved if the time spent occupying the lock is minimized. -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579458103 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: I understand your point. I try to use `Collections.synchronizedMap` and get the benchmark results as follows doc count | field cardinality | query point | baseline QPS | candidate QPS | diff percentage -- | -- | -- | -- | -- | -- 3000 | 10 | 1 | 2481 | 5102 | 105.6% 3000 | 10 | 1 | 2481 | 4843 | 95.2% (using `LongAdder `) 3000 | 10 | 1 | 2481 | 3851 | 55% (using `LongAdder ` And `ConcurrentHashMap`) 3000 | 10 | 1 | 2481 | 3751 | 51% (using `LongAdder ` And `Collections.synchronizedMap`) It seems that performance can be improved if the time spent occupying the lock is minimized. -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579596679 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: but I don't think it is the best to use `Collections.synchronizedMap`, we can use `ConcurrentLinkedDeque` to achieve LRU expiration better. Here is an example. ``` public class LRUCache { private final int capacity; private ConcurrentMap cache; private ConcurrentLinkedDeque keys; public LRUCache(int capacity) { this.capacity = capacity; this.cache = new ConcurrentHashMap<>(capacity); this.keys = new ConcurrentLinkedDeque<>(); } public V get(K key) { V value = cache.get(key); if (value != null) { keys.remove(key); // remove first occurrence of key keys.addFirst(key); // add key to the head } return value; } public void put(K key, V value) { if (cache.containsKey(key)) { keys.remove(key); // remove first occurrence of key } while (keys.size() >= capacity) { K lastKey = keys.removeLast(); if (lastKey != null) { cache.remove(lastKey); } } cache.put(key, value); keys.addFirst(key); } } ``` -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579596679 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: but I don't think it is the best to use `Collections.synchronizedMap`, we can use `ConcurrentLinkedDeque` to achieve LRU expiration with better performance. Here is an example. ``` public class LRUCache { private final int capacity; private ConcurrentMap cache; private ConcurrentLinkedDeque keys; public LRUCache(int capacity) { this.capacity = capacity; this.cache = new ConcurrentHashMap<>(capacity); this.keys = new ConcurrentLinkedDeque<>(); } public V get(K key) { V value = cache.get(key); if (value != null) { keys.remove(key); // remove first occurrence of key keys.addFirst(key); // add key to the head } return value; } public void put(K key, V value) { if (cache.containsKey(key)) { keys.remove(key); // remove first occurrence of key } while (keys.size() >= capacity) { K lastKey = keys.removeLast(); if (lastKey != null) { cache.remove(lastKey); } } cache.put(key, value); keys.addFirst(key); } } ``` -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579458103 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: I understand your point. I try to use `Collections.synchronizedMap` and get the benchmark results as follows doc count | field cardinality | query point | baseline QPS | candidate QPS | diff percentage -- | -- | -- | -- | -- | -- 3000 | 10 | 1 | 2481 | 5102 | 105.6% 3000 | 10 | 1 | 2481 | 4843 | 95.2% (using `LongAdder `) 3000 | 10 | 1 | 2481 | 3851 | 55% (using `LongAdder ` And `ConcurrentHashMap`) 3000 | 10 | 1 | 2481 | 3751 | 51% (using `LongAdder ` And `Collections.synchronizedMap`) It seems that performance can be improved if the locking time is minimized. -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579458103 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: I understand your point. I try to use `Collections.synchronizedMap` and get the benchmark results as follows doc count | field cardinality | query point | baseline QPS | candidate QPS | diff percentage -- | -- | -- | -- | -- | -- 3000 | 10 | 1 | 2481 | 5102 | 105.6% 3000 | 10 | 1 | 2481 | 4843 | 95.2% (using `LongAdder `) 3000 | 10 | 1 | 2481 | 3851 | 55% (using `LongAdder ` And `ConcurrentHashMap`) 3000 | 10 | 1 | 2481 | 3751 | 51% (using `LongAdder ` And `Collections.synchronizedMap`) It seems that performance still can be improved if the locking time is minimized. -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579596679 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: but I don't think it is the best to use `Collections.synchronizedMap`, we can use `ConcurrentLinkedDeque` to achieve LRU expiration with better performance. Here is an example. ``` public class LRUCache { private final int capacity; private ConcurrentMap cache; private ConcurrentLinkedDeque keys; private final ReentrantReadWriteLock lock; public LRUCache(int capacity) { this.capacity = capacity; this.cache = new ConcurrentHashMap<>(capacity); this.keys = new ConcurrentLinkedDeque<>(); this.lock = new ReentrantReadWriteLock(); } public V get(K key) { lock.readLock().lock(); try { V value = cache.get(key); if (value != null) { keys.remove(key); // remove first occurrence of key keys.addFirst(key); // add key to the head } return value; } finally { lock.readLock().unlock(); } } public void put(K key, V value) { lock.writeLock().lock(); try { if (cache.containsKey(key)) { keys.remove(key); // remove first occurrence of key } while (keys.size() >= capacity) { K lastKey = keys.removeLast(); if (lastKey != null) { cache.remove(lastKey); } } cache.put(key, value); keys.addFirst(key); } finally { lock.writeLock().unlock(); } } } ``` -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579596679 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: but I don't think it is the best to use `Collections.synchronizedMap`, we can use a better structure. -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579596679 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: but I don't think it is the best to use `Collections.synchronizedMap`, I wonder if we can use a better structure as below.  -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579596679 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: but I wonder if we can use a better structure as below.  -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
benwtrent commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1579769524 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: switching from an LRU to an LFU would require more consideration. But, your debugging here shows that there is a fair amount of performance being left on the table. I am surprised to see how much of a bottle neck cache access actually is. -- 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] Improve int4 compressed comparisons performance [lucene]
ChrisHegarty commented on PR #13321: URL: https://github.com/apache/lucene/pull/13321#issuecomment-2077675140 > Because this makes things measurably slower. .. Ok. That's a very good reason! ;-) > And I am not sure how it makes the comparisons easier. We then have to add LoC to decompress both and then compare both. The iteration code, etc. all remains the same. Right. It's just that the masking and shifting would apply equally to each of the inputs. And we'd load less data - since the number of bytes in the query would be less. I'm perfectly ok with the code as it is, I was mainly ensuring that such was considered (which is was). -- 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: [I] Decouple within-query concurrency from the index's segment geometry [LUCENE-8675] [lucene]
msokolov commented on issue #9721: URL: https://github.com/apache/lucene/issues/9721#issuecomment-2078030793 I started to play around with a hacky implementation that: 1. adds docid interval [start, end] to `LeafReaderContext` 2. modifies IndexSearcher so it creates these "searcher" leaves and puts these in its slices instead of the reader's leaf contexts 3. wraps every query in an IntervalQuery whose scorer uses a BoundedDocIdSetIterator (we already have this elsewhere as an inner class; I just pulled it out to top level). I don't claim this is a good API design, but it mostly just works and I just wanted to do the least work possible to enable this using (mostly) existing APIs to see what the impact would be on our application. I found the initial implementation in Lucene to be pretty straightforward, but in our application it seems we tie a *lot* of things to reader leaves that now have to be tied to searcher leaves, and these tend to be stored in arrays based on the number of leaves which now would not be viable - I expect an IndexSearcher might want to dynamically vary the way it slices an index? I guess the main learning I have from that is that we do want to have an explicit API change reflecting this - otherwise it is easy to fall into traps where everything compiles and seems fine, but things are actually quite broken - eg formerly you expected only a single thread to be accessing your per-leaf data structures, but now multiple ones can. Also @stefanvodita pointed me to Luca's branch https:/ /github.com/javanna/lucene/commit/6ca5680c94452f02d38bd0a5ba881da9ae950bae which seems to have a somewhat cleaner API. -- 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: [I] Decouple within-query concurrency from the index's segment geometry [LUCENE-8675] [lucene]
msokolov commented on issue #9721: URL: https://github.com/apache/lucene/issues/9721#issuecomment-2078043920 but I guess as an execution strategy it kind of made sense to me -- is it really necessary to clone Scorers? Could we create new ones for each searcher-segment or do we think that is too costly due to initialization costs? -- 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] Add test for parsing brackets in range queries [lucene]
dweiss commented on PR #13323: URL: https://github.com/apache/lucene/pull/13323#issuecomment-2078091483 > Any thoughts on whether queryParser.parse( "[ a\\[i\\] TO b\\[i\\] ]" ); should work? I did play around with updating the grammar, it should be possible but I need to think more about how it might break backwards compatibility. Looking at the javacc file it seems range terms are parsed differently compared to normal tokens. This is strange but I bet there was a reason for it - now long forgotten. I think, logically, it should accept the same term notation that is used elsewhere. It will break backward compatibility but maybe consistency is worth this cost (for the major release)? -- 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] Improve int4 compressed comparisons performance [lucene]
rmuir commented on PR #13321: URL: https://github.com/apache/lucene/pull/13321#issuecomment-2078163094 panama logic looks good to me. Not for this PR, just a question, is it possible to use this same trick for 8 bit integer calculations too? I imagine this trick could probably help there as well, if it is safe. > // iterate in chunks of 1024 items to ensure we don't overflow the short accumulator -- 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] [LUCENE-13044][replicator] NRT add configurable commitData for Custom… [lucene]
github-actions[bot] commented on PR #13045: URL: https://github.com/apache/lucene/pull/13045#issuecomment-2078378730 This PR has not had activity in the past 2 weeks, labeling it as stale. If the PR is waiting for review, notify the d...@lucene.apache.org list. Thank you for your contribution! -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1580286445 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: Yes, So I want to try best to optimize this neck. I think `uniqueQueries` should be `Collections.synchronizedMap` to achieve LRU expiration, but `cache` can be `ConcurrentHashMap` to improve 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-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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1580286445 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: Yes, So I want to try my best to optimize this neck. I think `uniqueQueries` should be `Collections.synchronizedMap` to achieve LRU expiration, but `cache` can be `ConcurrentHashMap` to improve 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-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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1580286445 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: Yes, So I want to try my best to optimize this neck. I think `uniqueQueries` should be `Collections.synchronizedMap` to achieve LRU expiration, but `cache` does not need order and can be `ConcurrentHashMap` to improve 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-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] Binary search all terms. [lucene]
vsop-479 commented on code in PR #13192: URL: https://github.com/apache/lucene/pull/13192#discussion_r1543119650 ## lucene/core/src/java/org/apache/lucene/codecs/lucene90/blocktree/SegmentTermsEnumFrame.java: ## @@ -196,6 +207,90 @@ void loadBlock() throws IOException { suffixLengthsReader.reset(suffixLengthBytes, 0, numSuffixLengthBytes); totalSuffixBytes = ste.in.getFilePointer() - startSuffixFP; +// Prepare suffixes, offsets to binary search. +if (allEqual) { + if (isLeafBlock) { +suffix = suffixLengthsReader.readVInt(); + } else { +// Handle subCode for non leaf block. +postions = new int[entCount]; Review Comment: Yes, maybe it is less worth to implement this binary search to non-leaf blocks (both `allEqual` and non-`allEqual` ?). Can we just take this change to non-`allEqual` leaf blocks? But it still need to allocate `suffixes ` and `offsets` ( assume We don't need `positions` to set `suffixLengthsReader`'s position after searching a block). -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1580286445 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: Yes, So I want to try my best to optimize this neck. I think `uniqueQueries` should be `Collections.synchronizedMap` to achieve LRU expiration, but `cache` does not need order and can be `ConcurrentHashMap` to improve performance. doc count | field cardinality | query point | baseline QPS | candidate QPS | diff percentage | diff -- | -- | -- | -- | -- | -- | -- 3000 | 10 | 1 | 2481 | 5102 | 105.6% | 3000 | 10 | 1 | 2481 | 4843 | 95.2% | `LongAdder ` 3000 | 10 | 1 | 2481 | 3751 | 51%| `LongAdder ` , `uniqueQueries` to be `Collections.synchronizedMap` , 3000 | 10 | 1 | 2481 | 4571 | 84% | `LongAdder`, `uniqueQueries` to be`Collections.synchronizedMap` , `cache` to be`ConcurrentHashMap` -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1580286445 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: Yes, So I want to try my best to optimize this neck. I think `uniqueQueries` should be `Collections.synchronizedMap` to achieve LRU expiration, but `cache` does not need order and can be `ConcurrentHashMap` to improve performance. doc count | field cardinality | query point | baseline QPS | candidate QPS | diff percentage | diff -- | -- | -- | -- | -- | -- | -- 3000 | 10 | 1 | 2481 | 5102 | 105.6% | 3000 | 10 | 1 | 2481 | 4843 | 95.2% | `LongAdder ` 3000 | 10 | 1 | 2481 | 3751 | 51%| `LongAdder ` , `uniqueQueries` and `cache` to be `Collections.synchronizedMap` , 3000 | 10 | 1 | 2481 | 4571 | 84% | `LongAdder`, `uniqueQueries` to be`Collections.synchronizedMap` , `cache` to be`ConcurrentHashMap` -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1580286445 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: Yes, So I want to try my best to optimize this neck. I think `uniqueQueries` should be `Collections.synchronizedMap` to achieve LRU expiration, but `cache` does not need order and can be `ConcurrentHashMap` to improve performance. doc count | field cardinality | query point | baseline QPS | candidate QPS | diff percentage | diff -- | -- | -- | -- | -- | -- | -- 3000 | 10 | 1 | 2481 | 5102 | 105.6% | 3000 | 10 | 1 | 2481 | 4843 | 95.2% | using `LongAdder ` 3000 | 10 | 1 | 2481 | 3751 | 51%| `uniqueQueries` and `cache` to be `Collections.synchronizedMap` . 3000 | 10 | 1 | 2481 | 4571 | 84% | `uniqueQueries` to be`Collections.synchronizedMap` , `cache` to be`ConcurrentHashMap` -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1580286445 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: Yes, So I want to try my best to optimize this neck. I think `uniqueQueries` should be `Collections.synchronizedMap` to achieve LRU expiration, but `cache` does not need order and can be `ConcurrentHashMap` to improve performance. doc count | field cardinality | query point | baseline QPS | candidate QPS | diff percentage | diff -- | -- | -- | -- | -- | -- | -- 3000 | 10 | 1 | 2481 | 5102 | 105.6% | 3000 | 10 | 1 | 2481 | 4843 | 95.2% | using `LongAdder ` 3000 | 10 | 1 | 2481 | 3751 | 51%| `uniqueQueries` and `cache` to be `Collections.synchronizedMap`. using `LongAdder ` 3000 | 10 | 1 | 2481 | 4571 | 84% | `uniqueQueries` to be`Collections.synchronizedMap` , `cache` to be`ConcurrentHashMap`. using `LongAdder ` -- 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] Performance improvements to use read lock to access LRUQueryCache [lucene]
boicehuang commented on code in PR #13306: URL: https://github.com/apache/lucene/pull/13306#discussion_r1580286445 ## lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java: ## @@ -265,7 +269,6 @@ boolean requiresEviction() { } CacheAndCount get(Query key, IndexReader.CacheHelper cacheHelper) { -assert lock.isHeldByCurrentThread(); assert key instanceof BoostQuery == false; assert key instanceof ConstantScoreQuery == false; final IndexReader.CacheKey readerKey = cacheHelper.getKey(); Review Comment: Yes, So I want to try my best to optimize this neck. I think `uniqueQueries` should be `Collections.synchronizedMap` to achieve LRU expiration, but `cache` does not need order and can be `ConcurrentHashMap` to improve performance. doc count | field cardinality | query point | baseline QPS | candidate QPS | diff percentage | diff -- | -- | -- | -- | -- | -- | -- 3000 | 10 | 1 | 2481 | 5102 | 105.6% | 3000 | 10 | 1 | 2481 | 4843 | 95.2% | using `LongAdder` 3000 | 10 | 1 | 2481 | 3751 | 51%| `uniqueQueries` and `cache` to be `Collections.synchronizedMap`. using `LongAdder` 3000 | 10 | 1 | 2481 | 4571 | 84% | `uniqueQueries` to be`Collections.synchronizedMap`, `cache` to be`ConcurrentHashMap`. using `LongAdder` -- 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