[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-14 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r334583710
 
 

 ##
 File path: lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
 ##
 @@ -1691,4 +1964,184 @@ public void testBulkScorerLocking() throws Exception {
 t.start();
 t.join();
   }
+
+  public void testRejectedExecution() throws IOException {
+ExecutorService service = new TestIndexSearcher.RejectingMockExecutor();
+Directory dir = newDirectory();
+final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+Document doc = new Document();
+StringField f = new StringField("color", "blue", Store.NO);
+doc.add(f);
+w.addDocument(doc);
+f.setStringValue("red");
+w.addDocument(doc);
+f.setStringValue("green");
+w.addDocument(doc);
+final DirectoryReader reader = w.getReader();
+
+final Query red = new TermQuery(new Term("color", "red"));
+
+IndexSearcher searcher = new IndexSearcher(reader, service);
+
+final LRUQueryCache queryCache = new LRUQueryCache(2, 10, context -> 
true);
+
+searcher.setQueryCache(queryCache);
+searcher.setQueryCachingPolicy(ALWAYS_CACHE);
+
+// To ensure that failing ExecutorService still allows query to run
+// successfully
+
+searcher.search(new ConstantScoreQuery(red), 1);
+assertEquals(Collections.singletonList(red), queryCache.cachedQueries());
+
+reader.close();
+w.close();
+dir.close();
+service.shutdown();
+  }
+
+  public void testClosedReaderExecution() throws IOException {
+CountDownLatch latch = new CountDownLatch(1);
+ExecutorService service = new BlockedMockExecutor(latch);
+
+Directory dir = newDirectory();
+final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+for (int i = 0; i < 100; i++) {
+  Document doc = new Document();
+  StringField f = new StringField("color", "blue", Store.NO);
+  doc.add(f);
+  w.addDocument(doc);
+  f.setStringValue("red");
+  w.addDocument(doc);
+  f.setStringValue("green");
+  w.addDocument(doc);
+
+  if (i % 10 == 0) {
+w.commit();
+  }
+}
+
+final DirectoryReader reader = w.getReader();
+
+final Query red = new TermQuery(new Term("color", "red"));
+
+IndexSearcher searcher = new IndexSearcher(reader, service) {
+  @Override
+  protected LeafSlice[] slices(List leaves) {
+ArrayList slices = new ArrayList<>();
+for (LeafReaderContext ctx : leaves) {
+  slices.add(new LeafSlice(Arrays.asList(ctx)));
+}
+return slices.toArray(LeafSlice[]::new);
+  }
+};
+
+final LRUQueryCache queryCache = new LRUQueryCache(2, 10, context -> 
true);
+
+searcher.setQueryCache(queryCache);
+searcher.setQueryCachingPolicy(ALWAYS_CACHE);
+
+// To ensure that failing ExecutorService still allows query to run
+// successfully
+
+ExecutorService tempService = new ThreadPoolExecutor(2, 2, 0L, 
TimeUnit.MILLISECONDS,
+new LinkedBlockingQueue(),
+new NamedThreadFactory("TestLRUQueryCache"));
+
+tempService.submit(new Runnable() {
+  @Override
+  public void run() {
+try {
+  Thread.sleep(100);
+  List leaves = searcher.leafContexts;
+
+  for (LeafReaderContext leafReaderContext : leaves) {
+leafReaderContext.reader().close();
+  }
 
 Review comment:
   I'd rather do `searcher.getIndexReader().close()` than close at the leaf 
level directly


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-14 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r334581927
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -813,6 +840,19 @@ public BulkScorer bulkScorer(LeafReaderContext context) 
throws IOException {
 
   if (docIdSet == null) {
 if (policy.shouldCache(in.getQuery())) {
+  boolean cacheSynchronously = executor == null;
+  // If asynchronous caching is requested, perform the same and return
+  // the uncached iterator
+  if (cacheSynchronously == false) {
+boolean asyncCachingSucceeded = cacheAsynchronously(context, 
cacheHelper);
+
+// If async caching failed, we will perform synchronous caching
+// hence do not return the uncached value here
+if (asyncCachingSucceeded == false) {
 
 Review comment:
   ```suggestion
   if (asyncCachingSucceeded) {
   ```


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-14 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r334583334
 
 

 ##
 File path: lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
 ##
 @@ -1691,4 +1964,184 @@ public void testBulkScorerLocking() throws Exception {
 t.start();
 t.join();
   }
+
+  public void testRejectedExecution() throws IOException {
+ExecutorService service = new TestIndexSearcher.RejectingMockExecutor();
+Directory dir = newDirectory();
+final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+Document doc = new Document();
+StringField f = new StringField("color", "blue", Store.NO);
+doc.add(f);
+w.addDocument(doc);
+f.setStringValue("red");
+w.addDocument(doc);
+f.setStringValue("green");
+w.addDocument(doc);
+final DirectoryReader reader = w.getReader();
+
+final Query red = new TermQuery(new Term("color", "red"));
+
+IndexSearcher searcher = new IndexSearcher(reader, service);
+
+final LRUQueryCache queryCache = new LRUQueryCache(2, 10, context -> 
true);
+
+searcher.setQueryCache(queryCache);
+searcher.setQueryCachingPolicy(ALWAYS_CACHE);
+
+// To ensure that failing ExecutorService still allows query to run
+// successfully
+
+searcher.search(new ConstantScoreQuery(red), 1);
+assertEquals(Collections.singletonList(red), queryCache.cachedQueries());
+
+reader.close();
+w.close();
+dir.close();
+service.shutdown();
+  }
+
+  public void testClosedReaderExecution() throws IOException {
+CountDownLatch latch = new CountDownLatch(1);
+ExecutorService service = new BlockedMockExecutor(latch);
+
+Directory dir = newDirectory();
+final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+for (int i = 0; i < 100; i++) {
+  Document doc = new Document();
+  StringField f = new StringField("color", "blue", Store.NO);
+  doc.add(f);
+  w.addDocument(doc);
+  f.setStringValue("red");
+  w.addDocument(doc);
+  f.setStringValue("green");
+  w.addDocument(doc);
+
+  if (i % 10 == 0) {
+w.commit();
+  }
+}
+
+final DirectoryReader reader = w.getReader();
+
+final Query red = new TermQuery(new Term("color", "red"));
+
+IndexSearcher searcher = new IndexSearcher(reader, service) {
+  @Override
+  protected LeafSlice[] slices(List leaves) {
+ArrayList slices = new ArrayList<>();
+for (LeafReaderContext ctx : leaves) {
+  slices.add(new LeafSlice(Arrays.asList(ctx)));
+}
+return slices.toArray(LeafSlice[]::new);
+  }
+};
+
+final LRUQueryCache queryCache = new LRUQueryCache(2, 10, context -> 
true);
+
+searcher.setQueryCache(queryCache);
+searcher.setQueryCachingPolicy(ALWAYS_CACHE);
+
+// To ensure that failing ExecutorService still allows query to run
+// successfully
+
+ExecutorService tempService = new ThreadPoolExecutor(2, 2, 0L, 
TimeUnit.MILLISECONDS,
+new LinkedBlockingQueue(),
+new NamedThreadFactory("TestLRUQueryCache"));
+
+tempService.submit(new Runnable() {
+  @Override
+  public void run() {
+try {
+  Thread.sleep(100);
 
 Review comment:
   Why do we need to sleep? In general we aim at never sleeping in tests. The 
only exception you would find use sleep because it might find test failures 
that couldn't get exposed otherwise.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-14 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r334586442
 
 

 ##
 File path: lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
 ##
 @@ -1691,4 +1964,184 @@ public void testBulkScorerLocking() throws Exception {
 t.start();
 t.join();
   }
+
+  public void testRejectedExecution() throws IOException {
+ExecutorService service = new TestIndexSearcher.RejectingMockExecutor();
+Directory dir = newDirectory();
+final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+Document doc = new Document();
+StringField f = new StringField("color", "blue", Store.NO);
+doc.add(f);
+w.addDocument(doc);
+f.setStringValue("red");
+w.addDocument(doc);
+f.setStringValue("green");
+w.addDocument(doc);
+final DirectoryReader reader = w.getReader();
+
+final Query red = new TermQuery(new Term("color", "red"));
+
+IndexSearcher searcher = new IndexSearcher(reader, service);
+
+final LRUQueryCache queryCache = new LRUQueryCache(2, 10, context -> 
true);
+
+searcher.setQueryCache(queryCache);
+searcher.setQueryCachingPolicy(ALWAYS_CACHE);
+
+// To ensure that failing ExecutorService still allows query to run
+// successfully
+
+searcher.search(new ConstantScoreQuery(red), 1);
+assertEquals(Collections.singletonList(red), queryCache.cachedQueries());
+
+reader.close();
+w.close();
+dir.close();
+service.shutdown();
+  }
+
+  public void testClosedReaderExecution() throws IOException {
+CountDownLatch latch = new CountDownLatch(1);
+ExecutorService service = new BlockedMockExecutor(latch);
+
+Directory dir = newDirectory();
+final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+for (int i = 0; i < 100; i++) {
+  Document doc = new Document();
+  StringField f = new StringField("color", "blue", Store.NO);
+  doc.add(f);
+  w.addDocument(doc);
+  f.setStringValue("red");
+  w.addDocument(doc);
+  f.setStringValue("green");
+  w.addDocument(doc);
+
+  if (i % 10 == 0) {
+w.commit();
+  }
+}
+
+final DirectoryReader reader = w.getReader();
+
+final Query red = new TermQuery(new Term("color", "red"));
+
+IndexSearcher searcher = new IndexSearcher(reader, service) {
+  @Override
+  protected LeafSlice[] slices(List leaves) {
+ArrayList slices = new ArrayList<>();
+for (LeafReaderContext ctx : leaves) {
+  slices.add(new LeafSlice(Arrays.asList(ctx)));
+}
+return slices.toArray(LeafSlice[]::new);
+  }
+};
+
+final LRUQueryCache queryCache = new LRUQueryCache(2, 10, context -> 
true);
+
+searcher.setQueryCache(queryCache);
+searcher.setQueryCachingPolicy(ALWAYS_CACHE);
+
+// To ensure that failing ExecutorService still allows query to run
+// successfully
+
+ExecutorService tempService = new ThreadPoolExecutor(2, 2, 0L, 
TimeUnit.MILLISECONDS,
+new LinkedBlockingQueue(),
+new NamedThreadFactory("TestLRUQueryCache"));
+
+tempService.submit(new Runnable() {
+  @Override
+  public void run() {
+try {
+  Thread.sleep(100);
+  List leaves = searcher.leafContexts;
+
+  for (LeafReaderContext leafReaderContext : leaves) {
+leafReaderContext.reader().close();
+  }
+
+  reader.close();
+} catch (Exception e) {
+  throw new RuntimeException(e.getMessage());
+}
+
+latch.countDown();
+
+  }
+});
+
+expectThrows(AlreadyClosedException.class, () -> searcher.search(new 
ConstantScoreQuery(red), 1));
 
 Review comment:
   Actually the interesting case is that the search succeeds but the caching 
fails? Can we simulate this case? Maybe we could create a searcher with a 
single leaf to force IndexSearcher to run search on the current thread, so that 
we would know that the only thing that runs in the threadpool is the caching? 
Or maybe we could wrap the cache in order to force it to use a special 
BlockedMockExecutor instead of the Executor of the IndexSearcher?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-11 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r333896448
 
 

 ##
 File path: lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
 ##
 @@ -244,6 +275,213 @@ public void testLRUEviction() throws Exception {
 dir.close();
   }
 
+  public void testLRUConcurrentLoadAndEviction() throws Exception {
+Directory dir = newDirectory();
+final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+Document doc = new Document();
+StringField f = new StringField("color", "blue", Store.NO);
+doc.add(f);
+w.addDocument(doc);
+f.setStringValue("red");
+w.addDocument(doc);
+f.setStringValue("green");
+w.addDocument(doc);
+final DirectoryReader reader = w.getReader();
+ExecutorService service = new ThreadPoolExecutor(4, 4, 0L, 
TimeUnit.MILLISECONDS,
+new LinkedBlockingQueue(),
+new NamedThreadFactory("TestLRUQueryCache"));
+
+IndexSearcher searcher = new IndexSearcher(reader, service);
+
+final CountDownLatch[] latch = {new CountDownLatch(1)};
+
+final LRUQueryCache queryCache = new LRUQueryCache(2, 10, context -> 
true) {
+  @Override
+  protected void onDocIdSetCache(Object readerCoreKey, long ramBytesUsed) {
+super.onDocIdSetCache(readerCoreKey, ramBytesUsed);
+latch[0].countDown();
+  }
+};
+
+final Query blue = new TermQuery(new Term("color", "blue"));
+final Query red = new TermQuery(new Term("color", "red"));
+final Query green = new TermQuery(new Term("color", "green"));
+
+assertEquals(Collections.emptyList(), queryCache.cachedQueries());
+
+searcher.setQueryCache(queryCache);
+// the filter is not cached on any segment: no changes
+searcher.setQueryCachingPolicy(NEVER_CACHE);
+searcher.search(new ConstantScoreQuery(green), 1);
+assertEquals(Collections.emptyList(), queryCache.cachedQueries());
+
+searcher.setQueryCachingPolicy(ALWAYS_CACHE);
+
+// First read should miss
+searcher.search(new ConstantScoreQuery(red), 1);
+
+
+// Let the cache load be completed
+latch[0].await();
+searcher.search(new ConstantScoreQuery(red), 1);
 
 Review comment:
   I think we should assert that the hit count incremented, in addition to 
searching again?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-11 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r333892366
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -732,6 +734,21 @@ public ScorerSupplier scorerSupplier(LeafReaderContext 
context) throws IOExcepti
 
   if (docIdSet == null) {
 if (policy.shouldCache(in.getQuery())) {
+  boolean cacheSynchronously = executor == null;
+
+  // If asynchronous caching is requested, perform the same and return
+  // the uncached iterator
+  if (cacheSynchronously == false) {
+boolean asyncCachingSucceeded;
+asyncCachingSucceeded = cacheAsynchronously(context, cacheHelper);
 
 Review comment:
   merge declaration and assignment?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-11 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r333896563
 
 

 ##
 File path: lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
 ##
 @@ -244,6 +275,213 @@ public void testLRUEviction() throws Exception {
 dir.close();
   }
 
+  public void testLRUConcurrentLoadAndEviction() throws Exception {
+Directory dir = newDirectory();
+final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+Document doc = new Document();
+StringField f = new StringField("color", "blue", Store.NO);
+doc.add(f);
+w.addDocument(doc);
+f.setStringValue("red");
+w.addDocument(doc);
+f.setStringValue("green");
+w.addDocument(doc);
+final DirectoryReader reader = w.getReader();
+ExecutorService service = new ThreadPoolExecutor(4, 4, 0L, 
TimeUnit.MILLISECONDS,
+new LinkedBlockingQueue(),
+new NamedThreadFactory("TestLRUQueryCache"));
+
+IndexSearcher searcher = new IndexSearcher(reader, service);
+
+final CountDownLatch[] latch = {new CountDownLatch(1)};
+
+final LRUQueryCache queryCache = new LRUQueryCache(2, 10, context -> 
true) {
+  @Override
+  protected void onDocIdSetCache(Object readerCoreKey, long ramBytesUsed) {
+super.onDocIdSetCache(readerCoreKey, ramBytesUsed);
+latch[0].countDown();
+  }
+};
+
+final Query blue = new TermQuery(new Term("color", "blue"));
+final Query red = new TermQuery(new Term("color", "red"));
+final Query green = new TermQuery(new Term("color", "green"));
+
+assertEquals(Collections.emptyList(), queryCache.cachedQueries());
+
+searcher.setQueryCache(queryCache);
+// the filter is not cached on any segment: no changes
+searcher.setQueryCachingPolicy(NEVER_CACHE);
+searcher.search(new ConstantScoreQuery(green), 1);
+assertEquals(Collections.emptyList(), queryCache.cachedQueries());
+
+searcher.setQueryCachingPolicy(ALWAYS_CACHE);
+
+// First read should miss
+searcher.search(new ConstantScoreQuery(red), 1);
+
+
+// Let the cache load be completed
+latch[0].await();
+searcher.search(new ConstantScoreQuery(red), 1);
+
+// Second read should hit
+searcher.search(new ConstantScoreQuery(red), 1);
+assertEquals(Collections.singletonList(red), queryCache.cachedQueries());
+
+latch[0] = new CountDownLatch(1);
+searcher.search(new ConstantScoreQuery(green), 1);
+
+// Let the cache load be completed
+latch[0].await();
+assertEquals(Arrays.asList(red, green), queryCache.cachedQueries());
+
+searcher.search(new ConstantScoreQuery(red), 1);
+assertEquals(Arrays.asList(green, red), queryCache.cachedQueries());
 
 Review comment:
   Check that the hit count incremented?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-11 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r333894810
 
 

 ##
 File path: lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
 ##
 @@ -244,6 +275,213 @@ public void testLRUEviction() throws Exception {
 dir.close();
   }
 
+  public void testLRUConcurrentLoadAndEviction() throws Exception {
+Directory dir = newDirectory();
+final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+Document doc = new Document();
+StringField f = new StringField("color", "blue", Store.NO);
+doc.add(f);
+w.addDocument(doc);
+f.setStringValue("red");
+w.addDocument(doc);
+f.setStringValue("green");
+w.addDocument(doc);
+final DirectoryReader reader = w.getReader();
+ExecutorService service = new ThreadPoolExecutor(4, 4, 0L, 
TimeUnit.MILLISECONDS,
+new LinkedBlockingQueue(),
+new NamedThreadFactory("TestLRUQueryCache"));
+
+IndexSearcher searcher = new IndexSearcher(reader, service);
+
+final CountDownLatch[] latch = {new CountDownLatch(1)};
+
+final LRUQueryCache queryCache = new LRUQueryCache(2, 10, context -> 
true) {
+  @Override
+  protected void onDocIdSetCache(Object readerCoreKey, long ramBytesUsed) {
+super.onDocIdSetCache(readerCoreKey, ramBytesUsed);
+latch[0].countDown();
+  }
+};
+
+final Query blue = new TermQuery(new Term("color", "blue"));
+final Query red = new TermQuery(new Term("color", "red"));
+final Query green = new TermQuery(new Term("color", "green"));
+
+assertEquals(Collections.emptyList(), queryCache.cachedQueries());
+
+searcher.setQueryCache(queryCache);
+// the filter is not cached on any segment: no changes
+searcher.setQueryCachingPolicy(NEVER_CACHE);
+searcher.search(new ConstantScoreQuery(green), 1);
+assertEquals(Collections.emptyList(), queryCache.cachedQueries());
+
+searcher.setQueryCachingPolicy(ALWAYS_CACHE);
+
+// First read should miss
+searcher.search(new ConstantScoreQuery(red), 1);
+
+
+// Let the cache load be completed
+latch[0].await();
+searcher.search(new ConstantScoreQuery(red), 1);
+
+// Second read should hit
+searcher.search(new ConstantScoreQuery(red), 1);
+assertEquals(Collections.singletonList(red), queryCache.cachedQueries());
 
 Review comment:
   shouldn't we be able to assert on this directly after the call to 
`latch[0].await();` returns?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-11 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r333900618
 
 

 ##
 File path: lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
 ##
 @@ -1691,4 +1954,180 @@ public void testBulkScorerLocking() throws Exception {
 t.start();
 t.join();
   }
+
+  public void testRejectedExecution() throws IOException {
+ExecutorService service = new TestIndexSearcher.RejectingMockExecutor();
+Directory dir = newDirectory();
+final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+Document doc = new Document();
+StringField f = new StringField("color", "blue", Store.NO);
+doc.add(f);
+w.addDocument(doc);
+f.setStringValue("red");
+w.addDocument(doc);
+f.setStringValue("green");
+w.addDocument(doc);
+final DirectoryReader reader = w.getReader();
+
+final Query red = new TermQuery(new Term("color", "red"));
+
+IndexSearcher searcher = new IndexSearcher(reader, service);
+
+final LRUQueryCache queryCache = new LRUQueryCache(2, 10, context -> 
true);
+
+searcher.setQueryCache(queryCache);
+searcher.setQueryCachingPolicy(ALWAYS_CACHE);
+
+// To ensure that failing ExecutorService still allows query to run
+// successfully
+
+searcher.search(new ConstantScoreQuery(red), 1);
+assertEquals(Collections.singletonList(red), queryCache.cachedQueries());
+
+reader.close();
+w.close();
+dir.close();
+service.shutdown();
+  }
+
+  public void testClosedReaderExecution() throws IOException {
+CountDownLatch latch = new CountDownLatch(1);
+ExecutorService service = new BlockedMockExecutor(latch);
+
+Directory dir = newDirectory();
+final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+for (int i = 0; i < 100; i++) {
+  Document doc = new Document();
+  StringField f = new StringField("color", "blue", Store.NO);
+  doc.add(f);
+  w.addDocument(doc);
+  f.setStringValue("red");
+  w.addDocument(doc);
+  f.setStringValue("green");
+  w.addDocument(doc);
+
+  if (i % 10 == 0) {
+w.commit();
+  }
+}
+
+final DirectoryReader reader = w.getReader();
+
+final Query red = new TermQuery(new Term("color", "red"));
+
+IndexSearcher searcher = new IndexSearcher(reader, service) {
+  @Override
+  protected LeafSlice[] slices(List leaves) {
+ArrayList slices = new ArrayList<>();
+for (LeafReaderContext ctx : leaves) {
+  slices.add(new LeafSlice(Arrays.asList(ctx)));
+}
+return slices.toArray(new LeafSlice[0]);
 
 Review comment:
   nit: with recent versions of Java I like `slices.toArray(LeafSlice[]::new);` 
better


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-11 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r333892145
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -449,12 +452,8 @@ void assertConsistent() {
   }
 
   @Override
-  public Weight doCache(Weight weight, QueryCachingPolicy policy) {
-while (weight instanceof CachingWrapperWeight) {
-  weight = ((CachingWrapperWeight) weight).in;
-}
-
-return new CachingWrapperWeight(weight, policy);
+  public Weight doCache(final Weight weight, QueryCachingPolicy policy, 
Executor executor) {
+return new CachingWrapperWeight(weight, policy, executor);
 
 Review comment:
   should we keep the unwrapping?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-11 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r333898028
 
 

 ##
 File path: lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
 ##
 @@ -244,6 +275,213 @@ public void testLRUEviction() throws Exception {
 dir.close();
   }
 
+  public void testLRUConcurrentLoadAndEviction() throws Exception {
+Directory dir = newDirectory();
+final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+Document doc = new Document();
+StringField f = new StringField("color", "blue", Store.NO);
+doc.add(f);
+w.addDocument(doc);
+f.setStringValue("red");
+w.addDocument(doc);
+f.setStringValue("green");
+w.addDocument(doc);
+final DirectoryReader reader = w.getReader();
+ExecutorService service = new ThreadPoolExecutor(4, 4, 0L, 
TimeUnit.MILLISECONDS,
+new LinkedBlockingQueue(),
+new NamedThreadFactory("TestLRUQueryCache"));
+
+IndexSearcher searcher = new IndexSearcher(reader, service);
+
+final CountDownLatch[] latch = {new CountDownLatch(1)};
+
+final LRUQueryCache queryCache = new LRUQueryCache(2, 10, context -> 
true) {
+  @Override
+  protected void onDocIdSetCache(Object readerCoreKey, long ramBytesUsed) {
+super.onDocIdSetCache(readerCoreKey, ramBytesUsed);
+latch[0].countDown();
+  }
+};
+
+final Query blue = new TermQuery(new Term("color", "blue"));
+final Query red = new TermQuery(new Term("color", "red"));
+final Query green = new TermQuery(new Term("color", "green"));
+
+assertEquals(Collections.emptyList(), queryCache.cachedQueries());
+
+searcher.setQueryCache(queryCache);
+// the filter is not cached on any segment: no changes
+searcher.setQueryCachingPolicy(NEVER_CACHE);
+searcher.search(new ConstantScoreQuery(green), 1);
+assertEquals(Collections.emptyList(), queryCache.cachedQueries());
+
+searcher.setQueryCachingPolicy(ALWAYS_CACHE);
+
+// First read should miss
+searcher.search(new ConstantScoreQuery(red), 1);
+
+
+// Let the cache load be completed
+latch[0].await();
+searcher.search(new ConstantScoreQuery(red), 1);
+
+// Second read should hit
+searcher.search(new ConstantScoreQuery(red), 1);
+assertEquals(Collections.singletonList(red), queryCache.cachedQueries());
+
+latch[0] = new CountDownLatch(1);
+searcher.search(new ConstantScoreQuery(green), 1);
+
+// Let the cache load be completed
+latch[0].await();
+assertEquals(Arrays.asList(red, green), queryCache.cachedQueries());
+
+searcher.search(new ConstantScoreQuery(red), 1);
+assertEquals(Arrays.asList(green, red), queryCache.cachedQueries());
+
+latch[0] = new CountDownLatch(1);
+
+searcher.search(new ConstantScoreQuery(blue), 1);
+
+// Let the cache load be completed
+latch[0].await();
+assertEquals(Arrays.asList(red, blue), queryCache.cachedQueries());
+
+searcher.search(new ConstantScoreQuery(blue), 1);
+assertEquals(Arrays.asList(red, blue), queryCache.cachedQueries());
+
+latch[0] = new CountDownLatch(1);
+
+searcher.search(new ConstantScoreQuery(green), 1);
+
+// Let the cache load be completed
+latch[0].await();
+assertEquals(Arrays.asList(blue, green), queryCache.cachedQueries());
+
+searcher.setQueryCachingPolicy(NEVER_CACHE);
+searcher.search(new ConstantScoreQuery(red), 1);
+assertEquals(Arrays.asList(blue, green), queryCache.cachedQueries());
 
 Review comment:
   maybe move the call to service.shutdown() above this line and also call 
`awaitTermination` to make sure that any ongoing cache operation are done so 
that the assertion doesn't succeed only because we are lucky with timing?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-11 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r333898690
 
 

 ##
 File path: lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
 ##
 @@ -244,6 +275,213 @@ public void testLRUEviction() throws Exception {
 dir.close();
   }
 
+  public void testLRUConcurrentLoadAndEviction() throws Exception {
+Directory dir = newDirectory();
+final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+Document doc = new Document();
+StringField f = new StringField("color", "blue", Store.NO);
+doc.add(f);
+w.addDocument(doc);
+f.setStringValue("red");
+w.addDocument(doc);
+f.setStringValue("green");
+w.addDocument(doc);
+final DirectoryReader reader = w.getReader();
+ExecutorService service = new ThreadPoolExecutor(4, 4, 0L, 
TimeUnit.MILLISECONDS,
+new LinkedBlockingQueue(),
+new NamedThreadFactory("TestLRUQueryCache"));
+
+IndexSearcher searcher = new IndexSearcher(reader, service);
+
+final CountDownLatch[] latch = {new CountDownLatch(1)};
+
+final LRUQueryCache queryCache = new LRUQueryCache(2, 10, context -> 
true) {
+  @Override
+  protected void onDocIdSetCache(Object readerCoreKey, long ramBytesUsed) {
+super.onDocIdSetCache(readerCoreKey, ramBytesUsed);
+latch[0].countDown();
+  }
+};
+
+final Query blue = new TermQuery(new Term("color", "blue"));
+final Query red = new TermQuery(new Term("color", "red"));
+final Query green = new TermQuery(new Term("color", "green"));
+
+assertEquals(Collections.emptyList(), queryCache.cachedQueries());
+
+searcher.setQueryCache(queryCache);
+// the filter is not cached on any segment: no changes
+searcher.setQueryCachingPolicy(NEVER_CACHE);
+searcher.search(new ConstantScoreQuery(green), 1);
+assertEquals(Collections.emptyList(), queryCache.cachedQueries());
+
+searcher.setQueryCachingPolicy(ALWAYS_CACHE);
+
+// First read should miss
+searcher.search(new ConstantScoreQuery(red), 1);
+
+
+// Let the cache load be completed
+latch[0].await();
+searcher.search(new ConstantScoreQuery(red), 1);
+
+// Second read should hit
+searcher.search(new ConstantScoreQuery(red), 1);
+assertEquals(Collections.singletonList(red), queryCache.cachedQueries());
+
+latch[0] = new CountDownLatch(1);
+searcher.search(new ConstantScoreQuery(green), 1);
+
+// Let the cache load be completed
+latch[0].await();
+assertEquals(Arrays.asList(red, green), queryCache.cachedQueries());
+
+searcher.search(new ConstantScoreQuery(red), 1);
+assertEquals(Arrays.asList(green, red), queryCache.cachedQueries());
+
+latch[0] = new CountDownLatch(1);
+
+searcher.search(new ConstantScoreQuery(blue), 1);
+
+// Let the cache load be completed
+latch[0].await();
+assertEquals(Arrays.asList(red, blue), queryCache.cachedQueries());
+
+searcher.search(new ConstantScoreQuery(blue), 1);
+assertEquals(Arrays.asList(red, blue), queryCache.cachedQueries());
+
+latch[0] = new CountDownLatch(1);
+
+searcher.search(new ConstantScoreQuery(green), 1);
+
+// Let the cache load be completed
+latch[0].await();
+assertEquals(Arrays.asList(blue, green), queryCache.cachedQueries());
+
+searcher.setQueryCachingPolicy(NEVER_CACHE);
+searcher.search(new ConstantScoreQuery(red), 1);
+assertEquals(Arrays.asList(blue, green), queryCache.cachedQueries());
+
+reader.close();
+w.close();
+dir.close();
+service.shutdown();
+  }
+
+  public void testLRUConcurrentLoadsOfSameQuery() throws Exception {
+Directory dir = newDirectory();
+final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+Document doc = new Document();
+StringField f = new StringField("color", "blue", Store.NO);
+doc.add(f);
+w.addDocument(doc);
+f.setStringValue("red");
+w.addDocument(doc);
+f.setStringValue("green");
+w.addDocument(doc);
+final DirectoryReader reader = w.getReader();
+ExecutorService service = new ThreadPoolExecutor(4, 4, 0L, 
TimeUnit.MILLISECONDS,
+new LinkedBlockingQueue(),
+new NamedThreadFactory("TestLRUQueryCache"));
+
+ExecutorService stressService = new ThreadPoolExecutor(15, 15, 0L, 
TimeUnit.MILLISECONDS,
+new LinkedBlockingQueue(),
+new NamedThreadFactory("TestLRUQueryCache2"));
+
+IndexSearcher searcher = new IndexSearcher(reader, service);
+
+final CountDownLatch latch = new CountDownLatch(1);
+
+final LRUQueryCache queryCache = new LRUQueryCache(2, 10, context -> 
true) {
+  @Override
+  protected void onDocIdSetCache(Object readerCoreKey, long ramBytesUsed) {
+super.onDocIdSetCache(readerCoreKey, ramBytesUsed);
+latch.countDown();
+  }
+};
+
+final Query green 

[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-11 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r333907193
 
 

 ##
 File path: lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
 ##
 @@ -1691,4 +1954,180 @@ public void testBulkScorerLocking() throws Exception {
 t.start();
 t.join();
   }
+
+  public void testRejectedExecution() throws IOException {
+ExecutorService service = new TestIndexSearcher.RejectingMockExecutor();
+Directory dir = newDirectory();
+final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+Document doc = new Document();
+StringField f = new StringField("color", "blue", Store.NO);
+doc.add(f);
+w.addDocument(doc);
+f.setStringValue("red");
+w.addDocument(doc);
+f.setStringValue("green");
+w.addDocument(doc);
+final DirectoryReader reader = w.getReader();
+
+final Query red = new TermQuery(new Term("color", "red"));
+
+IndexSearcher searcher = new IndexSearcher(reader, service);
+
+final LRUQueryCache queryCache = new LRUQueryCache(2, 10, context -> 
true);
+
+searcher.setQueryCache(queryCache);
+searcher.setQueryCachingPolicy(ALWAYS_CACHE);
+
+// To ensure that failing ExecutorService still allows query to run
+// successfully
+
+searcher.search(new ConstantScoreQuery(red), 1);
+assertEquals(Collections.singletonList(red), queryCache.cachedQueries());
+
+reader.close();
+w.close();
+dir.close();
+service.shutdown();
+  }
+
+  public void testClosedReaderExecution() throws IOException {
+CountDownLatch latch = new CountDownLatch(1);
+ExecutorService service = new BlockedMockExecutor(latch);
+
+Directory dir = newDirectory();
+final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+for (int i = 0; i < 100; i++) {
+  Document doc = new Document();
+  StringField f = new StringField("color", "blue", Store.NO);
+  doc.add(f);
+  w.addDocument(doc);
+  f.setStringValue("red");
+  w.addDocument(doc);
+  f.setStringValue("green");
+  w.addDocument(doc);
+
+  if (i % 10 == 0) {
+w.commit();
+  }
+}
+
+final DirectoryReader reader = w.getReader();
+
+final Query red = new TermQuery(new Term("color", "red"));
+
+IndexSearcher searcher = new IndexSearcher(reader, service) {
+  @Override
+  protected LeafSlice[] slices(List leaves) {
+ArrayList slices = new ArrayList<>();
+for (LeafReaderContext ctx : leaves) {
+  slices.add(new LeafSlice(Arrays.asList(ctx)));
+}
+return slices.toArray(new LeafSlice[0]);
+  }
+};
+
+final LRUQueryCache queryCache = new LRUQueryCache(2, 10, context -> 
true);
+
+searcher.setQueryCache(queryCache);
+searcher.setQueryCachingPolicy(ALWAYS_CACHE);
+
+// To ensure that failing ExecutorService still allows query to run
+// successfully
+
+ExecutorService tempService = new ThreadPoolExecutor(2, 2, 0L, 
TimeUnit.MILLISECONDS,
+new LinkedBlockingQueue(),
+new NamedThreadFactory("TestLRUQueryCache"));
+
+tempService.submit(new Runnable() {
+  @Override
+  public void run() {
+try {
+  Thread.sleep(100);
+  reader.close();
+} catch (Exception e) {
+  throw new RuntimeException(e.getMessage());
+}
+
+latch.countDown();
+
+  }
+});
+
+searcher.search(new ConstantScoreQuery(red), 1);
+
+assertEquals(Collections.singletonList(red), queryCache.cachedQueries());
 
 Review comment:
   This assertion is actually proving that the test is not working? We would 
except that nothing gets cached since the reader is already closed by the time 
that the executor needs to cache the query?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-09 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r332858863
 
 

 ##
 File path: lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
 ##
 @@ -244,6 +272,225 @@ public void testLRUEviction() throws Exception {
 dir.close();
   }
 
+  public void testLRUConcurrentLoadAndEviction() throws Exception {
+Directory dir = newDirectory();
+final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+Document doc = new Document();
+StringField f = new StringField("color", "blue", Store.NO);
+doc.add(f);
+w.addDocument(doc);
+f.setStringValue("red");
+w.addDocument(doc);
+f.setStringValue("green");
+w.addDocument(doc);
+final DirectoryReader reader = w.getReader();
+ExecutorService service = new ThreadPoolExecutor(4, 4, 0L, 
TimeUnit.MILLISECONDS,
+new LinkedBlockingQueue(),
+new NamedThreadFactory("TestLRUQueryCache"));
+
+IndexSearcher searcher = new IndexSearcher(reader, service);
+
+final CountDownLatch[] latch = {new CountDownLatch(1)};
+
+final LRUQueryCache queryCache = new LRUQueryCache(2, 10, context -> 
true) {
+  @Override
+  protected void onDocIdSetCache(Object readerCoreKey, long ramBytesUsed) {
+super.onDocIdSetCache(readerCoreKey, ramBytesUsed);
+latch[0].countDown();
+  }
+};
+
+final Query blue = new TermQuery(new Term("color", "blue"));
+final Query red = new TermQuery(new Term("color", "red"));
+final Query green = new TermQuery(new Term("color", "green"));
+
+assertEquals(Collections.emptyList(), queryCache.cachedQueries());
+
+searcher.setQueryCache(queryCache);
+// the filter is not cached on any segment: no changes
+searcher.setQueryCachingPolicy(NEVER_CACHE);
+searcher.search(new ConstantScoreQuery(green), 1);
+assertEquals(Collections.emptyList(), queryCache.cachedQueries());
+
+searcher.setQueryCachingPolicy(ALWAYS_CACHE);
+
+// First read should miss
+searcher.search(new ConstantScoreQuery(red), 1);
+
+if (!(queryCache.cachedQueries().equals(Collections.emptyList( {
+  searcher.search(new ConstantScoreQuery(red), 1);
+} else {
 
 Review comment:
   what is the motivation for running code under the `if` statement instead of 
always running the `else` 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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-09 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r332853732
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -732,8 +734,25 @@ public ScorerSupplier scorerSupplier(LeafReaderContext 
context) throws IOExcepti
 
   if (docIdSet == null) {
 if (policy.shouldCache(in.getQuery())) {
-  docIdSet = cache(context);
-  putIfAbsent(in.getQuery(), docIdSet, cacheHelper);
+  boolean cacheSynchronously = executor == null;
+  boolean asyncCachingSucceeded = false;
+
+  // If asynchronous caching is requested, perform the same and return
+  // the uncached iterator
+  if (cacheSynchronously == false) {
+asyncCachingSucceeded = cacheAsynchronously(context, cacheHelper);
+
+// If async caching failed, synchronous caching will
+// be performed, hence do not return the uncached value
+if (asyncCachingSucceeded) {
+  return in.scorerSupplier(context);
+}
+  }
+
+  if (cacheSynchronously || asyncCachingSucceeded == false) {
 
 Review comment:
   `asyncCachingSucceeded` can only be `false` here, or do I miss something? If 
this is correct, then we could declare `asyncCachingSucceeded` under the `if` 
statement to reduce its scope?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-09 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r332853116
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -832,5 +866,26 @@ public BulkScorer bulkScorer(LeafReaderContext context) 
throws IOException {
   return new DefaultBulkScorer(new ConstantScoreScorer(this, 0f, 
ScoreMode.COMPLETE_NO_SCORES, disi));
 }
 
+// Perform a cache load asynchronously
+// @return true if asynchronous caching succeeded, false otherwise
+private boolean cacheAsynchronously(LeafReaderContext context, 
IndexReader.CacheHelper cacheHelper) {
+  FutureTask task = new FutureTask<>(() -> {
+// If the reader is being closed -- do nothing
+if (context.reader().tryIncRef()) {
+  DocIdSet localDocIdSet = cache(context);
+  putIfAbsent(in.getQuery(), localDocIdSet, cacheHelper);
+}
 
 Review comment:
   It should decrease the reference counter in a finally block, e.g.
   
   ```java
   if (context.reader().tryIncRef()) {
 try {
   DocIdSet localDocIdSet = cache(context);
   putIfAbsent(in.getQuery(), localDocIdSet, cacheHelper);
 } finally {
   context.reader().decRef();
 }
   }
   ```


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-09 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r332854251
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -832,5 +866,26 @@ public BulkScorer bulkScorer(LeafReaderContext context) 
throws IOException {
   return new DefaultBulkScorer(new ConstantScoreScorer(this, 0f, 
ScoreMode.COMPLETE_NO_SCORES, disi));
 }
 
+// Perform a cache load asynchronously
+// @return true if asynchronous caching succeeded, false otherwise
+private boolean cacheAsynchronously(LeafReaderContext context, 
IndexReader.CacheHelper cacheHelper) {
+  FutureTask task = new FutureTask<>(() -> {
+// If the reader is being closed -- do nothing
 
 Review comment:
   ++


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-09 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r332857726
 
 

 ##
 File path: lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
 ##
 @@ -244,6 +272,225 @@ public void testLRUEviction() throws Exception {
 dir.close();
   }
 
+  public void testLRUConcurrentLoadAndEviction() throws Exception {
+Directory dir = newDirectory();
+final RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+
+Document doc = new Document();
+StringField f = new StringField("color", "blue", Store.NO);
+doc.add(f);
+w.addDocument(doc);
+f.setStringValue("red");
+w.addDocument(doc);
+f.setStringValue("green");
+w.addDocument(doc);
+final DirectoryReader reader = w.getReader();
+ExecutorService service = new ThreadPoolExecutor(4, 4, 0L, 
TimeUnit.MILLISECONDS,
+new LinkedBlockingQueue(),
+new NamedThreadFactory("TestLRUQueryCache"));
+
+IndexSearcher searcher = new IndexSearcher(reader, service);
+
+final CountDownLatch[] latch = {new CountDownLatch(1)};
+
+final LRUQueryCache queryCache = new LRUQueryCache(2, 10, context -> 
true) {
+  @Override
+  protected void onDocIdSetCache(Object readerCoreKey, long ramBytesUsed) {
+super.onDocIdSetCache(readerCoreKey, ramBytesUsed);
+latch[0].countDown();
+  }
+};
+
+final Query blue = new TermQuery(new Term("color", "blue"));
+final Query red = new TermQuery(new Term("color", "red"));
+final Query green = new TermQuery(new Term("color", "green"));
+
+assertEquals(Collections.emptyList(), queryCache.cachedQueries());
+
+searcher.setQueryCache(queryCache);
+// the filter is not cached on any segment: no changes
+searcher.setQueryCachingPolicy(NEVER_CACHE);
+searcher.search(new ConstantScoreQuery(green), 1);
+assertEquals(Collections.emptyList(), queryCache.cachedQueries());
+
+searcher.setQueryCachingPolicy(ALWAYS_CACHE);
+
+// First read should miss
+searcher.search(new ConstantScoreQuery(red), 1);
+
+if (!(queryCache.cachedQueries().equals(Collections.emptyList( {
+  searcher.search(new ConstantScoreQuery(red), 1);
+} else {
+  // Let the cache load be completed
+  latch[0].await(200, TimeUnit.MILLISECONDS);
 
 Review comment:
   I'd probably call `await()` without a timeout. The first reason is that CI 
environments are sometimes very busy and even some simple operations can take 
very long. The second is that if we get a timeout, we'll have no idea whether 
the query took too long to cache or whether it was something else. On the other 
hand, if we wait indefinitely, then the test runner will hit a timeout, and 
when it does so it emits a thread dump in order to understand what the threads 
were doing at the time of the timeout. So we'd have more information to 
understand what happened. I'd suggest removing all timeouts in the below 
await() calls.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-09 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r332859431
 
 

 ##
 File path: lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
 ##
 @@ -1229,17 +1476,49 @@ public void testMinSegmentSizePredicate() throws 
IOException {
 LRUQueryCache cache = new LRUQueryCache(2, 1, new 
LRUQueryCache.MinSegmentSizePredicate(2, 0f));
 searcher.setQueryCache(cache);
 searcher.count(new DummyQuery());
+
 assertEquals(0, cache.getCacheCount());
 
-cache = new LRUQueryCache(2, 1, new 
LRUQueryCache.MinSegmentSizePredicate(1, 0f));
+final CountDownLatch[] latch = { new CountDownLatch(1)};
+cache = new LRUQueryCache(2, 1,
+new LRUQueryCache.MinSegmentSizePredicate(1, 0f)) {
+@Override
+protected void onDocIdSetCache(Object readerCoreKey, long 
ramBytesUsed) {
+  super.onDocIdSetCache(readerCoreKey, ramBytesUsed);
+  latch[0].countDown();
+}
+  };
+
 searcher.setQueryCache(cache);
+
 searcher.count(new DummyQuery());
-assertEquals(1, cache.getCacheCount());
 
-cache = new LRUQueryCache(2, 1, new 
LRUQueryCache.MinSegmentSizePredicate(0, .6f));
+if (cache.getCacheCount() != 1) {
+  try {
+latch[0].await(200, TimeUnit.MILLISECONDS);
+  } catch (InterruptedException e) {
+throw new RuntimeException(e.getMessage());
+  }
+  assertEquals(1, cache.getCacheCount());
+}
 
 Review comment:
   should we always wait on the latch instead of first checking whether the 
cache count is 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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-09 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r332854167
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -813,8 +832,23 @@ public BulkScorer bulkScorer(LeafReaderContext context) 
throws IOException {
 
   if (docIdSet == null) {
 if (policy.shouldCache(in.getQuery())) {
-  docIdSet = cache(context);
-  putIfAbsent(in.getQuery(), docIdSet, cacheHelper);
+  boolean cacheSynchronously = executor == null;
+  // If asynchronous caching is requested, perform the same and return
+  // the uncached iterator
+  if (cacheSynchronously == false) {
+cacheSynchronously = cacheAsynchronously(context, cacheHelper);
 
 Review comment:
   we need to update this similarly to what we did with `scorerSupplier`?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-04 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r331589273
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -732,8 +734,25 @@ public ScorerSupplier scorerSupplier(LeafReaderContext 
context) throws IOExcepti
 
   if (docIdSet == null) {
 if (policy.shouldCache(in.getQuery())) {
-  docIdSet = cache(context);
-  putIfAbsent(in.getQuery(), docIdSet, cacheHelper);
+  boolean cacheSynchronously = executor == null;
+  boolean asyncCachingSucceeded = false;
+
+  // If asynchronous caching is requested, perform the same and return
+  // the uncached iterator
+  if (cacheSynchronously == false) {
+asyncCachingSucceeded = cacheAsynchronously(context, cacheHelper);
+
+// If async caching failed, synchronous caching will
+// be performed, hence do not return the uncached value
+if (asyncCachingSucceeded) {
+  return in.scorerSupplier(context);
+}
+  }
+
+  if (cacheSynchronously || !asyncCachingSucceeded) {
 
 Review comment:
   can you do == false for consistency?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-04 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r331588407
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/IndexSearcher.java
 ##
 @@ -181,7 +181,16 @@ public IndexSearcher(IndexReader r) {
   }
 
   /** Runs searches for each segment separately, using the
-   *  provided Executor. NOTE:
+   *  provided Executor. The passed in Executor will also be
+   *  used by LRUQueryCache (if enabled) to perform asynchronous
+   *  query caching.
+   *  If a task is rejected by the host Executor, the failed task
+   *  will then be executed on the caller thread. This is done to
+   *  ensure that a query succeeds, albeit with a higher latency.
+   *  If a user wishes to modify the said behaviour, they can either
+   *  handle the exception in the provided Executor, or override
+   *  the said method in a custom extension of IndexSearcher.
 
 Review comment:
   it's unclear to me which method you are recommending overriding?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-04 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r331589988
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -832,5 +866,23 @@ public BulkScorer bulkScorer(LeafReaderContext context) 
throws IOException {
   return new DefaultBulkScorer(new ConstantScoreScorer(this, 0f, 
ScoreMode.COMPLETE_NO_SCORES, disi));
 }
 
+// Perform a cache load asynchronously
+// @return true if asynchronous caching succeeded, false otherwise
+private boolean cacheAsynchronously(LeafReaderContext context, 
IndexReader.CacheHelper cacheHelper) {
+  FutureTask task = new FutureTask<>(() -> {
+DocIdSet localDocIdSet = cache(context);
 
 Review comment:
   Since we are caching asynchronously, we might me hitting a closed segment. 
We should use `LeafReader#tryIncRef` to make sure the reader is still open and 
add tests for that case.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-04 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r331387075
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -832,5 +873,23 @@ public BulkScorer bulkScorer(LeafReaderContext context) 
throws IOException {
   return new DefaultBulkScorer(new ConstantScoreScorer(this, 0f, 
ScoreMode.COMPLETE_NO_SCORES, disi));
 }
 
+// Perform a cache load asynchronously
+// @return true if synchronous caching is needed, false otherwise
 
 Review comment:
   I think this contract is a bit confusing. Usually `true` would indicate 
success, like `Lock#tryLock`. Maybe it would be less confusing if `true` meant 
that we successfully triggered an async cache operation?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-04 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r331385596
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -450,11 +453,15 @@ void assertConsistent() {
 
   @Override
   public Weight doCache(Weight weight, QueryCachingPolicy policy) {
-while (weight instanceof CachingWrapperWeight) {
-  weight = ((CachingWrapperWeight) weight).in;
-}
+return doCache(weight, policy, null /* executor */);
+  }
 
-return new CachingWrapperWeight(weight, policy);
+  // Should be used only when the user wishes to trade throughput for latency
+  // This method was not merged in the method above as to not break the 
existing contract
+  // advertised by QueryCache
 
 Review comment:
   I think breaking is fine since QueryCache is `@lucene.experimental`.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-04 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r331393345
 
 

 ##
 File path: lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
 ##
 @@ -1229,17 +1476,53 @@ public void testMinSegmentSizePredicate() throws 
IOException {
 LRUQueryCache cache = new LRUQueryCache(2, 1, new 
LRUQueryCache.MinSegmentSizePredicate(2, 0f));
 searcher.setQueryCache(cache);
 searcher.count(new DummyQuery());
+
 assertEquals(0, cache.getCacheCount());
 
-cache = new LRUQueryCache(2, 1, new 
LRUQueryCache.MinSegmentSizePredicate(1, 0f));
+final CountDownLatch[] latch = { new CountDownLatch(1)};
+cache = new LRUQueryCache(2, 1,
+new LRUQueryCache.MinSegmentSizePredicate(1, 0f)) {
+@Override
+protected void onDocIdSetCache(Object readerCoreKey, long 
ramBytesUsed) {
+  super.onDocIdSetCache(readerCoreKey, ramBytesUsed);
+  latch[0].countDown();
+}
+  };
+
 searcher.setQueryCache(cache);
+
 searcher.count(new DummyQuery());
-assertEquals(1, cache.getCacheCount());
 
-cache = new LRUQueryCache(2, 1, new 
LRUQueryCache.MinSegmentSizePredicate(0, .6f));
+if (cache.getCacheCount() != 1) {
+  try {
+latch[0].await(200, TimeUnit.MILLISECONDS);
+  } catch (InterruptedException e) {
+throw new RuntimeException(e.getMessage());
+  }
+  assertEquals(1, cache.getCacheCount());
+}
+
+latch[0] = new CountDownLatch(1);
+cache = new LRUQueryCache(2, 1,
+new LRUQueryCache.MinSegmentSizePredicate(0, .6f)) {
+  @Override
+  protected void onDocIdSetCache(Object readerCoreKey, long ramBytesUsed) {
+super.onDocIdSetCache(readerCoreKey, ramBytesUsed);
+latch[0].countDown();
+  }
+};
+
 searcher.setQueryCache(cache);
+
 searcher.count(new DummyQuery());
-assertEquals(1, cache.getCacheCount());
+if (cache.getCacheCount() != 1) {
+  try {
+latch[0].await(200, TimeUnit.MILLISECONDS);
+  } catch (InterruptedException e) {
+throw new RuntimeException(e.getMessage());
 
 Review comment:
   Can we avoid swallowing the exception?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-03 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r331190828
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -88,13 +92,47 @@
  * @lucene.experimental
  */
 public class LRUQueryCache implements QueryCache, Accountable {
+  /** Act as key for the inflight queries map */
+  private static class MapKey {
+private final Query query;
+private final IndexReader.CacheKey cacheKey;
+
+public MapKey(Query query, IndexReader.CacheKey cacheKey) {
+  this.query = query;
+  this.cacheKey = cacheKey;
+}
+
+public Query getQuery() {
+  return query;
+}
+
+public IndexReader.CacheKey getCacheKey() {
+  return cacheKey;
+}
+
+@Override
+public int hashCode() { return query.hashCode() ^ cacheKey.hashCode(); }
 
 Review comment:
   Can you use the usual polynomial formula `h0 + 31 * (h1 + 31 * ( ... ))`. 
The use of `^` is fine here, but it is a source of performance bugs in some 
scenarios. For instance Java's AbstractMap.Entry does this, which means that 
whenever you map a key to itself, the hashcode is 0 (because both hashcodes are 
equal). This is a lot of collisions if you have lots of entries that have the 
same key and value. So I prefer avoiding this pattern.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-03 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r331191644
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -732,8 +821,24 @@ public ScorerSupplier scorerSupplier(LeafReaderContext 
context) throws IOExcepti
 
   if (docIdSet == null) {
 if (policy.shouldCache(in.getQuery())) {
-  docIdSet = cache(context);
-  putIfAbsent(in.getQuery(), docIdSet, cacheHelper);
+  boolean cacheSynchronously = executor == null;
+
+  // If asynchronous caching is requested, perform the same and return
+  // the uncached iterator
+  if (cacheSynchronously == false) {
+cacheSynchronously = cacheAsynchronously(context, cacheHelper);
+
+// If async caching failed, synchronous caching will
+// be performed, hence do not return the uncached value
+if (cacheSynchronously == false) {
+  return in.scorerSupplier(context);
+}
+  }
+
+  if (cacheSynchronously) {
 
 Review comment:
   Thanks I had missed it.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-03 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r331191303
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -656,10 +713,21 @@ public long ramBytesUsed() {
 // threads when IndexSearcher is created with threads
 private final AtomicBoolean used;
 
+private final Executor executor;
+
 CachingWrapperWeight(Weight in, QueryCachingPolicy policy) {
   super(in.getQuery(), 1f);
   this.in = in;
   this.policy = policy;
+  this.executor = null;
+  used = new AtomicBoolean(false);
+}
+
+CachingWrapperWeight(Weight in, QueryCachingPolicy policy, Executor 
executor) {
+  super(in.getQuery(), 1f);
+  this.in = in;
+  this.policy = policy;
+  this.executor = executor;
   used = new AtomicBoolean(false);
 }
 
 Review comment:
   nit: can we have a single constructor? The first one looks unused?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-03 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r331192596
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -832,5 +931,43 @@ public BulkScorer bulkScorer(LeafReaderContext context) 
throws IOException {
   return new DefaultBulkScorer(new ConstantScoreScorer(this, 0f, 
ScoreMode.COMPLETE_NO_SCORES, disi));
 }
 
+// Perform a cache load asynchronously
+// @return true if synchronous caching is needed, false otherwise
+private boolean cacheAsynchronously(LeafReaderContext context, 
IndexReader.CacheHelper cacheHelper) {
+  /*
+   * If the current query is already being asynchronously cached,
+   * do not trigger another cache operation
+   */
+  if (inFlightAsyncLoadQueries.add(new MapKey(in.getQuery(),
+  cacheHelper.getKey())) == false) {
+return false;
+  }
+
+  FutureTask task = new FutureTask<>(() -> {
+try {
+  DocIdSet localDocIdSet = cache(context);
+  putIfAbsent(in.getQuery(), localDocIdSet, cacheHelper);
+} finally {
+  // Remove the key from inflight
+  Object retValue = inFlightAsyncLoadQueries.remove(new 
MapKey(in.getQuery(), cacheHelper.getKey()));
 
 Review comment:
   this call is protected by no lock while `inFlightAsyncLoadQueries` is not 
concurrent?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-02 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r330550479
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/QueryCache.java
 ##
 @@ -33,4 +35,10 @@
*/
   Weight doCache(Weight weight, QueryCachingPolicy policy);
 
+  /**
+   * Same as above, but allows passing in an Executor to perform caching
+   * asynchronously
+   */
+  Weight doCache(Weight weight, QueryCachingPolicy policy, Executor executor);
 
 Review comment:
   Let's remove the other doCache and only have this one, with a `null` 
executor signaling that things should get cached in the current thread?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-02 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r330554876
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -88,13 +93,36 @@
  * @lucene.experimental
  */
 public class LRUQueryCache implements QueryCache, Accountable {
+  /** Act as key for the inflight queries map */
+  private static class MapKey {
+private final Query query;
+private final IndexReader.CacheKey cacheKey;
+
+public MapKey(Query query, IndexReader.CacheKey cacheKey) {
+  this.query = query;
+  this.cacheKey = cacheKey;
+}
+
+public Query getQuery() {
+  return query;
+}
+
+public IndexReader.CacheKey getCacheKey() {
+  return cacheKey;
+}
+  }
 
 Review comment:
   We need equals/hashcode, or this will never prevent the caching of the same 
query multiple times.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-02 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r330564331
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -832,5 +952,47 @@ public BulkScorer bulkScorer(LeafReaderContext context) 
throws IOException {
   return new DefaultBulkScorer(new ConstantScoreScorer(this, 0f, 
ScoreMode.COMPLETE_NO_SCORES, disi));
 }
 
+// Perform a cache load asynchronously
+// @return true if synchronous caching is needed, false otherwise
+private boolean cacheAsynchronously(LeafReaderContext context, 
IndexReader.CacheHelper cacheHelper) {
+  /*
+   * If the current query is already being asynchronously cached,
+   * do not trigger another cache operation
+   */
+  Object returnValue = inFlightAsyncLoadQueries.putIfAbsent(new 
MapKey(in.getQuery(),
+  cacheHelper.getKey()), cacheHelper.getKey());
+
+  assert returnValue == null || returnValue == cacheHelper.getKey();
+
+  if (returnValue != null) {
+return false;
+  }
+
+  FutureTask task = new FutureTask<>(() -> {
+DocIdSet localDocIdSet = cache(context);
+putIfAbsent(in.getQuery(), localDocIdSet, cacheHelper);
+
+// Remove the key from inflight -- the key is loaded now
+Object retValue = inFlightAsyncLoadQueries.remove(new 
MapKey(in.getQuery(), cacheHelper.getKey()));
+
+// The query should have been present in the inflight queries set 
before
+// we actually loaded it -- hence the removal of the key should be 
successful
+assert retValue != null;
+
+if (countDownLatch != null) {
+  countDownLatch.countDown();
+}
+
+return null;
+  });
+  try {
+executor.execute(task);
+  } catch (RejectedExecutionException e) {
+// Trigger synchronous caching
+return true;
+  }
 
 Review comment:
   Same here, we need to remove from inFlightAsyncLoadQueries on every code 
path.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-02 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r330553473
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -389,6 +431,7 @@ public void clear() {
   cache.clear();
   // Note that this also clears the uniqueQueries map since 
mostRecentlyUsedQueries is the uniqueQueries.keySet view:
   mostRecentlyUsedQueries.clear();
+  inFlightAsyncLoadQueries.clear();
 
 Review comment:
   same here, I don't think it's correct?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-02 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r330546874
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -813,8 +918,23 @@ public BulkScorer bulkScorer(LeafReaderContext context) 
throws IOException {
 
   if (docIdSet == null) {
 if (policy.shouldCache(in.getQuery())) {
-  docIdSet = cache(context);
-  putIfAbsent(in.getQuery(), docIdSet, cacheHelper);
+  boolean cacheSynchronously = executor == null;
+  // If asynchronous caching is requested, perform the same and return
+  // the uncached iterator
+  if (cacheSynchronously == false) {
+cacheSynchronously = cacheAsynchronously(context, cacheHelper);
+
+// If async caching failed, we will perform synchronous caching
+// hence do not return the uncached value here
+if (cacheSynchronously == false) {
 
 Review comment:
   cacheSynchronously is necessarily false already?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-02 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r330572346
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -448,13 +491,48 @@ void assertConsistent() {
 }
   }
 
+  // pkg-private for testing
+  void setCountDownLatch(CountDownLatch latch) {
 
 Review comment:
   do you think we could avoid setting a latch here, and maybe instead calling 
countDown from a subclass' onDocIdSetCache?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-02 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r330552818
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -368,10 +400,20 @@ public void clearQuery(Query query) {
 onEviction(singleton);
   }
 } finally {
+  removeQuery(query);
 
 Review comment:
   I don't think this is correct? The fact that we are removing entries for a 
query doesn't cancel the loading of cache entries?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-02 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r330549857
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -832,5 +952,47 @@ public BulkScorer bulkScorer(LeafReaderContext context) 
throws IOException {
   return new DefaultBulkScorer(new ConstantScoreScorer(this, 0f, 
ScoreMode.COMPLETE_NO_SCORES, disi));
 }
 
+// Perform a cache load asynchronously
+// @return true if synchronous caching is needed, false otherwise
+private boolean cacheAsynchronously(LeafReaderContext context, 
IndexReader.CacheHelper cacheHelper) {
+  /*
+   * If the current query is already being asynchronously cached,
+   * do not trigger another cache operation
+   */
+  Object returnValue = inFlightAsyncLoadQueries.putIfAbsent(new 
MapKey(in.getQuery(),
+  cacheHelper.getKey()), cacheHelper.getKey());
+
+  assert returnValue == null || returnValue == cacheHelper.getKey();
+
+  if (returnValue != null) {
+return false;
+  }
+
+  FutureTask task = new FutureTask<>(() -> {
+DocIdSet localDocIdSet = cache(context);
+putIfAbsent(in.getQuery(), localDocIdSet, cacheHelper);
+
+// Remove the key from inflight -- the key is loaded now
+Object retValue = inFlightAsyncLoadQueries.remove(new 
MapKey(in.getQuery(), cacheHelper.getKey()));
 
 Review comment:
   We should probably put it in a finally block to make sure it runs even in 
case of exceptions in above calls. Otherwise we'd have a memory leak.


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [lucene-solr] jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous Caching in LRUQueryCache

2019-10-02 Thread GitBox
jpountz commented on a change in pull request #916: LUCENE-8213: Asynchronous 
Caching in LRUQueryCache
URL: https://github.com/apache/lucene-solr/pull/916#discussion_r330545241
 
 

 ##
 File path: lucene/core/src/java/org/apache/lucene/search/LRUQueryCache.java
 ##
 @@ -88,13 +93,36 @@
  * @lucene.experimental
  */
 public class LRUQueryCache implements QueryCache, Accountable {
+  /** Act as key for the inflight queries map */
+  private static class MapKey {
+private final Query query;
+private final IndexReader.CacheKey cacheKey;
+
+public MapKey(Query query, IndexReader.CacheKey cacheKey) {
+  this.query = query;
+  this.cacheKey = cacheKey;
+}
+
+public Query getQuery() {
+  return query;
+}
+
+public IndexReader.CacheKey getCacheKey() {
+  return cacheKey;
+}
+  }
 
   private final int maxSize;
   private final long maxRamBytesUsed;
   private final Predicate leavesToCache;
   // maps queries that are contained in the cache to a singleton so that this
   // cache does not store several copies of the same query
   private final Map uniqueQueries;
+  // Marks the inflight queries that are being asynchronously loaded into the 
cache
+  // This is used to ensure that multiple threads do not trigger loading
+  // of the same query in the same cache. We use a set because it is an 
invariant that
+  // the entries of this data structure be unique.
+  private final Map inFlightAsyncLoadQueries = 
new HashMap<>();
 
 Review comment:
   use a set instead? I see you added a couple of assertions on return values, 
but they don't seem to add more value than what we could get with a set?


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.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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