You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2022/02/14 14:35:30 UTC

[GitHub] [lucene] mogui opened a new pull request #679: Monitor Improvements LUCENE-10422

mogui opened a new pull request #679:
URL: https://github.com/apache/lucene/pull/679


   
   # Description
   
   Change how Monitor manages it's directory, IndexWriter and IndexReader, giving the possibility to
   - use a custom Directory implementation
   - use a readonly QueryIndex in order to have more Monitor instance on different server reading from the same index (now the index reader is created from the index writer so it is impossible to make a readonly Monitor)
   
   # Solution
   I've extended MonitorConfiguration, with a readonly boolean property that governs ho the QueryIndex is created, and I also added a functional property that acts as a Directory Provider
   
   # Tests
   
   I've added tests to confirm two readonly monitor could work as expected.
   
   # Checklist
   
   Please review the following and check all that apply:
   
   - [x] I have reviewed the guidelines for [How to Contribute](https://wiki.apache.org/lucene/HowToContribute) and my code conforms to the standards described there to the best of my ability.
   - [x] I have created a Jira issue and added the issue ID to my pull request title.
   - [ ] I have given Lucene maintainers [access](https://help.github.com/en/articles/allowing-changes-to-a-pull-request-branch-created-from-a-fork) to contribute to my PR branch. (optional but recommended)
   - [x] I have developed this patch against the `main` branch.
   - [x] I have run `./gradlew check`.
   - [x] I have added tests for my changes.
   


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

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

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



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


[GitHub] [lucene] romseygeek commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex extends QueryIndex {
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.queries = new HashMap<>();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+    this.populateQueryCache(serializer, decomposer);
+  }
+
+  @Override
+  public void commit(List<MonitorQuery> updates) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot commit");
+  }
+
+  @Override
+  long search(final Query query, QueryCollector matcher) throws IOException {
+    QueryBuilder builder = termFilter -> query;
+    return search(builder, matcher);
+  }
+
+  @Override
+  public long search(QueryBuilder queryBuilder, QueryCollector matcher) throws IOException {
+    IndexSearcher searcher = null;
+    try {
+      searcher = manager.acquire();
+      return searchInMemory(queryBuilder, matcher, searcher, this.queries);
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+  }
+
+  @Override
+  public void purgeCache() throws IOException {
+    this.populateQueryCache(serializer, decomposer);
+    lastPurged = System.nanoTime();
+  }
+
+  @Override
+  void purgeCache(CachePopulator populator) throws IOException {
+    manager.maybeRefresh();

Review comment:
       There's a race here, I think.  Once you've called `manager.maybeRefresh()` then a subsequent call to `search` will return the new searcher, but you may not have updated `this.queries` yet.  




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

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

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



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


[GitHub] [lucene] romseygeek commented on pull request #679: Monitor Improvements LUCENE-10422

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


   One last thing, can you add a CHANGES.txt entry in the Lucene 9.2.0 New Features section? And then I think we're good to merge! Thanks for all your patience on this.


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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex extends QueryIndex {
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.queries = new HashMap<>();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+    this.populateQueryCache(serializer, decomposer);
+  }
+
+  @Override
+  public void commit(List<MonitorQuery> updates) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot commit");
+  }
+
+  @Override
+  long search(final Query query, QueryCollector matcher) throws IOException {
+    QueryBuilder builder = termFilter -> query;
+    return search(builder, matcher);
+  }
+
+  @Override
+  public long search(QueryBuilder queryBuilder, QueryCollector matcher) throws IOException {
+    IndexSearcher searcher = null;
+    try {
+      searcher = manager.acquire();
+      return searchInMemory(queryBuilder, matcher, searcher, this.queries);
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+  }
+
+  @Override
+  public void purgeCache() throws IOException {
+    this.populateQueryCache(serializer, decomposer);
+    lastPurged = System.nanoTime();
+  }
+
+  @Override
+  void purgeCache(CachePopulator populator) throws IOException {
+    manager.maybeRefresh();

Review comment:
       @romseygeek didi had a chanche to look out the new commits?




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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex extends QueryIndex {
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.queries = new HashMap<>();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+    this.populateQueryCache(serializer, decomposer);
+  }
+
+  @Override
+  public void commit(List<MonitorQuery> updates) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot commit");
+  }
+
+  @Override
+  long search(final Query query, QueryCollector matcher) throws IOException {
+    QueryBuilder builder = termFilter -> query;
+    return search(builder, matcher);
+  }
+
+  @Override
+  public long search(QueryBuilder queryBuilder, QueryCollector matcher) throws IOException {
+    IndexSearcher searcher = null;
+    try {
+      searcher = manager.acquire();
+      return searchInMemory(queryBuilder, matcher, searcher, this.queries);
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+  }
+
+  @Override
+  public void purgeCache() throws IOException {
+    this.populateQueryCache(serializer, decomposer);
+    lastPurged = System.nanoTime();
+  }
+
+  @Override
+  void purgeCache(CachePopulator populator) throws IOException {
+    manager.maybeRefresh();

Review comment:
       @romseygeek did you had a chanche to look out the new commits?




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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.*;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex implements QueryIndex {
+  private final SearcherManager manager;
+  private final QueryDecomposer decomposer;
+  private final MonitorQuerySerializer serializer;
+
+  final Map<IndexReader.CacheKey, WritableQueryIndex.QueryTermFilter> termFilters = new HashMap<>();
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();

Review comment:
       Ah, good to know! what do you think it's best to do? Make sense what I have proposed?




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

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

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



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


[GitHub] [lucene] romseygeek commented on pull request #679: Monitor Improvements LUCENE-10422

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


   @mogui there isn't a roadmap for the monitor itself, but please do open issues and/or PRs if you have ideas for other improvements!


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

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

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



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


[GitHub] [lucene] romseygeek merged pull request #679: Monitor Improvements LUCENE-10422

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


   


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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/MonitorConfiguration.java
##########
@@ -47,16 +49,39 @@ private static IndexWriterConfig defaultIndexWriterConfig() {
     return iwc;
   }
 
+  public Boolean isReadOnly() {

Review comment:
       sure
   




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

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

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



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


[GitHub] [lucene] mogui commented on pull request #679: Monitor Improvements LUCENE-10422

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


   @romseygeek I should have fixed everything, also added few lines of docs to explain read-only behaviour.


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

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

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



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


[GitHub] [lucene] romseygeek commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex extends QueryIndex {
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.queries = new HashMap<>();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+    this.populateQueryCache(serializer, decomposer);
+  }
+
+  @Override
+  public void commit(List<MonitorQuery> updates) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot commit");
+  }
+
+  @Override
+  long search(final Query query, QueryCollector matcher) throws IOException {
+    QueryBuilder builder = termFilter -> query;
+    return search(builder, matcher);
+  }
+
+  @Override
+  public long search(QueryBuilder queryBuilder, QueryCollector matcher) throws IOException {
+    IndexSearcher searcher = null;
+    try {
+      searcher = manager.acquire();
+      return searchInMemory(queryBuilder, matcher, searcher, this.queries);
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+  }
+
+  @Override
+  public void purgeCache() throws IOException {
+    this.populateQueryCache(serializer, decomposer);
+    lastPurged = System.nanoTime();
+  }
+
+  @Override
+  void purgeCache(CachePopulator populator) throws IOException {
+    manager.maybeRefresh();

Review comment:
       I think actually the best solution is to remove the query cache entirely for this impl, which is where you started out - sorry for all the back and forth here.  We can have a background thread that calls maybeRefresh() on the manager to keep up with updates, but all the queries will be read directly from the searcher and parsed as they are executed.  The in-memory cache works when the Monitor in question is handling updates as well, but trying to do that when you have no idea what the changes are between IndexReaders is going to be nasty.




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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex extends QueryIndex {
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.queries = new HashMap<>();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+    this.populateQueryCache(serializer, decomposer);
+  }
+
+  @Override
+  public void commit(List<MonitorQuery> updates) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot commit");
+  }
+
+  @Override
+  long search(final Query query, QueryCollector matcher) throws IOException {
+    QueryBuilder builder = termFilter -> query;
+    return search(builder, matcher);
+  }
+
+  @Override
+  public long search(QueryBuilder queryBuilder, QueryCollector matcher) throws IOException {
+    IndexSearcher searcher = null;
+    try {
+      searcher = manager.acquire();
+      return searchInMemory(queryBuilder, matcher, searcher, this.queries);
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+  }
+
+  @Override
+  public void purgeCache() throws IOException {
+    this.populateQueryCache(serializer, decomposer);
+    lastPurged = System.nanoTime();
+  }
+
+  @Override
+  void purgeCache(CachePopulator populator) throws IOException {
+    manager.maybeRefresh();

Review comment:
       @romseygeek I just realized that main branch from which I started the fork requires java 17, is there a way to make a build that targets Java 11 or do I have to remake the pull request on topo of 9_0_0 branch ?




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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/QueryIndex.java
##########
@@ -87,8 +81,20 @@
   final Map<IndexReader.CacheKey, QueryTermFilter> termFilters = new HashMap<>();
 
   QueryIndex(MonitorConfiguration config, Presearcher presearcher) throws IOException {

Review comment:
       Nevermind, I got it or maybe I just think i did :)
   
   Tomorrow I'll update the pull request.




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

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

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



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


[GitHub] [lucene] romseygeek commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/MonitorConfiguration.java
##########
@@ -35,8 +34,9 @@
   private long purgeFrequency = 5;
   private TimeUnit purgeFrequencyUnits = TimeUnit.MINUTES;
   private QueryDecomposer queryDecomposer = new QueryDecomposer();
-  private Path indexPath = null;
   private MonitorQuerySerializer serializer;
+  private Boolean readOnly = false;
+  private DirectoryProviderFunctionalInterface directoryProvider;

Review comment:
       We can set this to a default value of `() -> new ByteBuffersDirectory()` which would then simplify the `buildIndexWriter()` implementation.

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/DirectoryProviderFunctionalInterface.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import org.apache.lucene.store.Directory;
+
+/** A functional interface to provide a Directory instance */
+public interface DirectoryProviderFunctionalInterface {

Review comment:
       Can we use `org.apache.lucene.util.IOSupplier` here instead please?

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/QueryIndex.java
##########
@@ -87,8 +81,20 @@
   final Map<IndexReader.CacheKey, QueryTermFilter> termFilters = new HashMap<>();
 
   QueryIndex(MonitorConfiguration config, Presearcher presearcher) throws IOException {

Review comment:
       Given that QueryIndex is package-private I think we have some leeway to split things up here a bit and make things clearer.  Could you try making QueryIndex itself an interface, and then having read-only and writeable implementations?  We could also move the purge executor from the Monitor into the writeable QueryIndex implementation as it's not necessary to clear the cache at all on a read-only query index.

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/QueryIndex.java
##########
@@ -36,15 +36,7 @@
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.StringField;
-import org.apache.lucene.index.BinaryDocValues;

Review comment:
       The code formatter will complain about these imports I think




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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.*;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex implements QueryIndex {
+  private final SearcherManager manager;
+  private final QueryDecomposer decomposer;
+  private final MonitorQuerySerializer serializer;
+
+  final Map<IndexReader.CacheKey, WritableQueryIndex.QueryTermFilter> termFilters = new HashMap<>();
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+  }
+
+  @Override
+  public void commit(List<MonitorQuery> updates) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot commit");
+  }
+
+  @Override
+  public MonitorQuery getQuery(String queryId) throws IOException {
+    if (serializer == null) {
+      throw new IllegalStateException(
+          "Cannot get queries from an index with no MonitorQuerySerializer");
+    }
+    BytesRef[] bytesHolder = new BytesRef[1];
+    search(
+        new TermQuery(new Term(WritableQueryIndex.FIELDS.query_id, queryId)),

Review comment:
       yes it' something that I thought about, I'll drop the interface for an  abstract class




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

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

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



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


[GitHub] [lucene] romseygeek commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.*;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex implements QueryIndex {
+  private final SearcherManager manager;
+  private final QueryDecomposer decomposer;
+  private final MonitorQuerySerializer serializer;
+
+  final Map<IndexReader.CacheKey, WritableQueryIndex.QueryTermFilter> termFilters = new HashMap<>();
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();

Review comment:
       At the moment you wouldn't pick up any changes anyway, because you're not calling `maybeRefresh()` on the SearcherManager so you would always get the same view of the index.  If you want a dynamic view then you'll need  a background refresh 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.

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.*;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex implements QueryIndex {
+  private final SearcherManager manager;
+  private final QueryDecomposer decomposer;
+  private final MonitorQuerySerializer serializer;
+
+  final Map<IndexReader.CacheKey, WritableQueryIndex.QueryTermFilter> termFilters = new HashMap<>();
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();

Review comment:
       Ok it's more clear now.
   I can do it that way, but The readonly monitor would need a way to repopulate the cache too, assuming there are other writer that insert and delete on the same index, the readonly would never get the delta untile it gets re-instantiated to populate its Map. 
   This way would not be so useful.




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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.*;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex implements QueryIndex {
+  private final SearcherManager manager;
+  private final QueryDecomposer decomposer;
+  private final MonitorQuerySerializer serializer;
+
+  final Map<IndexReader.CacheKey, WritableQueryIndex.QueryTermFilter> termFilters = new HashMap<>();
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();

Review comment:
       Or maybe restore alle the query cache logic in the abstract class and selectively choose if use it or not for both implementations?




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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex extends QueryIndex {
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+    this.populateQueryCache(serializer, decomposer);
+  }
+
+  @Override
+  public void commit(List<MonitorQuery> updates) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot commit");
+  }
+
+  @Override
+  long search(final Query query, QueryCollector matcher) throws IOException {
+    QueryBuilder builder = termFilter -> query;
+    return search(builder, matcher);
+  }
+
+  @Override
+  public long search(QueryBuilder queryBuilder, QueryCollector matcher) throws IOException {
+    IndexSearcher searcher = null;
+    try {
+      searcher = manager.acquire();
+      return searchInMemory(queryBuilder, matcher, searcher, this.queries);
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+  }
+
+  @Override
+  void purgeCache(CachePopulator populator) throws IOException {
+    final ConcurrentMap<String, QueryCacheEntry> newCache = new ConcurrentHashMap<>();

Review comment:
       True, but then we have to assign iot to `queries` that it is in the abstract class and it is 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.

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/test/org/apache/lucene/monitor/TestMonitorReadonly.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Collections;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.IndexNotFoundException;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.FSDirectory;
+import org.junit.Test;
+
+public class TestMonitorReadonly extends MonitorTestBase {
+  private static final Analyzer ANALYZER = new WhitespaceAnalyzer();
+
+  @Test
+  public void testReadonlyMonitorThrowsOnInexistentIndex() {
+    Path indexDirectory = createTempDir();
+    MonitorConfiguration config =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse),
+                true);
+    assertThrows(
+        IndexNotFoundException.class,
+        () -> {
+          new Monitor(ANALYZER, config);
+        });
+  }
+
+  @Test
+  public void testReadonlyMonitorThrowsWhenCallingWriteRequests() throws IOException {
+    Path indexDirectory = createTempDir();
+    MonitorConfiguration writeConfig =
+        new MonitorConfiguration()
+            .setIndexPath(
+                indexDirectory, MonitorQuerySerializer.fromParser(MonitorTestBase::parse));
+
+    // this will create the index
+    Monitor writeMonitor = new Monitor(ANALYZER, writeConfig);
+    writeMonitor.close();
+
+    MonitorConfiguration config =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse),
+                true);
+    try (Monitor monitor = new Monitor(ANALYZER, config)) {
+      assertThrows(
+          IllegalStateException.class,
+          () -> {
+            TermQuery query = new TermQuery(new Term(FIELD, "test"));
+            monitor.register(
+                new MonitorQuery("query1", query, query.toString(), Collections.emptyMap()));
+          });
+
+      assertThrows(
+          IllegalStateException.class,
+          () -> {
+            monitor.deleteById("query1");
+          });
+
+      assertThrows(
+          IllegalStateException.class,
+          () -> {
+            monitor.clear();
+          });
+    }
+  }
+
+  @Test
+  public void testSettingCustomDirectory() throws IOException {
+    Path indexDirectory = createTempDir();
+    Document doc = new Document();
+    doc.add(newTextField(FIELD, "This is a Foobar test document", Field.Store.NO));
+
+    MonitorConfiguration writeConfig =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse));
+
+    try (Monitor writeMonitor = new Monitor(ANALYZER, writeConfig)) {
+      TermQuery query = new TermQuery(new Term(FIELD, "test"));
+      writeMonitor.register(
+          new MonitorQuery("query1", query, query.toString(), Collections.emptyMap()));
+      TermQuery query2 = new TermQuery(new Term(FIELD, "Foobar"));
+      writeMonitor.register(
+          new MonitorQuery("query2", query2, query.toString(), Collections.emptyMap()));
+      MatchingQueries<QueryMatch> matches = writeMonitor.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertNotNull(matches.getMatches());
+      assertEquals(2, matches.getMatchCount());
+      assertNotNull(matches.matches("query2"));
+    }
+  }
+
+  public void testMonitorReadOnlyCouldReadOnTheSameIndex() throws IOException {
+    Path indexDirectory = createTempDir();
+    Document doc = new Document();
+    doc.add(newTextField(FIELD, "This is a test document", Field.Store.NO));
+
+    MonitorConfiguration writeConfig =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse));
+
+    try (Monitor writeMonitor = new Monitor(ANALYZER, writeConfig)) {
+      TermQuery query = new TermQuery(new Term(FIELD, "test"));
+      writeMonitor.register(
+          new MonitorQuery("query1", query, query.toString(), Collections.emptyMap()));
+    }
+
+    MonitorConfiguration readConfig =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse),
+                true);
+
+    try (Monitor readMonitor1 = new Monitor(ANALYZER, readConfig)) {
+      MatchingQueries<QueryMatch> matches = readMonitor1.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertNotNull(matches.getMatches());
+      assertEquals(1, matches.getMatchCount());
+      assertNotNull(matches.matches("query1"));
+    }
+
+    try (Monitor readMonitor2 = new Monitor(ANALYZER, readConfig)) {
+      MatchingQueries<QueryMatch> matches = readMonitor2.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertNotNull(matches.getMatches());
+      assertEquals(1, matches.getMatchCount());
+      assertNotNull(matches.matches("query1"));
+
+      assertThrows(
+          IllegalStateException.class,
+          () -> {
+            TermQuery query = new TermQuery(new Term(FIELD, "test"));
+            readMonitor2.register(
+                new MonitorQuery("query1", query, query.toString(), Collections.emptyMap()));
+          });
+    }
+  }

Review comment:
       Hi @romseygeek I am doing it, but indeed I cannot make it work, I am missing something?
   No change in monitor before purging it's ok, but after a purge still no changes, I wasn' calling maybeRefresh on the searchManager but also calling it before purging, I have still satale search results. What am I doing wrong ?




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

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

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



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


[GitHub] [lucene] romseygeek commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.*;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.NamedThreadFactory;
+
+class ReadonlyQueryIndex extends QueryIndex {
+
+  private final ScheduledExecutorService refreshExecutor;
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+    this.refreshExecutor =
+        Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("cache-purge"));
+    long refreshFrequency = configuration.getPurgeFrequency();
+    this.refreshExecutor.scheduleAtFixedRate(
+        () -> {
+          try {
+            manager.maybeRefresh();
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        },
+        refreshFrequency,
+        refreshFrequency,
+        configuration.getPurgeFrequencyUnits());
+  }
+
+  @Override
+  public void commit(List<MonitorQuery> updates) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot commit");
+  }
+
+  @Override
+  public long search(QueryBuilder queryBuilder, QueryCollector matcher) throws IOException {
+    IndexSearcher searcher = null;
+    try {
+      searcher = manager.acquire();
+      LazyMonitorQueryCollector collector =
+          new LazyMonitorQueryCollector(matcher, serializer, decomposer);
+      long buildTime = System.nanoTime();
+      Query query =
+          queryBuilder.buildQuery(
+              termFilters.get(searcher.getIndexReader().getReaderCacheHelper().getKey()));
+      buildTime = System.nanoTime() - buildTime;
+      searcher.search(query, collector);
+      return buildTime;
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+  }
+
+  @Override
+  public void purgeCache() {
+    throw new IllegalStateException("Monitor is readOnly, it has no cache");

Review comment:
       Let's make this call `manager.maybeRefresh`

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.*;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.NamedThreadFactory;
+
+class ReadonlyQueryIndex extends QueryIndex {
+
+  private final ScheduledExecutorService refreshExecutor;
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+    this.refreshExecutor =
+        Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("cache-purge"));
+    long refreshFrequency = configuration.getPurgeFrequency();
+    this.refreshExecutor.scheduleAtFixedRate(
+        () -> {
+          try {
+            manager.maybeRefresh();
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        },
+        refreshFrequency,
+        refreshFrequency,
+        configuration.getPurgeFrequencyUnits());
+  }
+
+  @Override
+  public void commit(List<MonitorQuery> updates) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot commit");
+  }
+
+  @Override
+  public long search(QueryBuilder queryBuilder, QueryCollector matcher) throws IOException {
+    IndexSearcher searcher = null;
+    try {
+      searcher = manager.acquire();
+      LazyMonitorQueryCollector collector =
+          new LazyMonitorQueryCollector(matcher, serializer, decomposer);
+      long buildTime = System.nanoTime();
+      Query query =
+          queryBuilder.buildQuery(
+              termFilters.get(searcher.getIndexReader().getReaderCacheHelper().getKey()));
+      buildTime = System.nanoTime() - buildTime;
+      searcher.search(query, collector);
+      return buildTime;
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+  }
+
+  @Override
+  public void purgeCache() {
+    throw new IllegalStateException("Monitor is readOnly, it has no cache");
+  }
+
+  @Override
+  void purgeCache(CachePopulator populator) {
+    throw new IllegalStateException("Monitor is readOnly, it has no cache");
+  }
+
+  @Override
+  public void close() throws IOException {
+    refreshExecutor.shutdown();
+    IOUtils.close(manager);
+  }
+
+  @Override
+  public int numDocs() throws IOException {
+    IndexSearcher searcher = null;
+    int numDocs;
+    try {
+      searcher = manager.acquire();
+      numDocs = searcher.getIndexReader().numDocs();
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+    return numDocs;
+  }
+
+  @Override
+  public int cacheSize() {
+    return -1;
+  }
+
+  @Override
+  public void deleteQueries(List<String> ids) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot delete queries");
+  }
+
+  @Override
+  public void clear() throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot clear");
+  }
+
+  @Override
+  public long getLastPurged() {
+    return -1;
+  }
+
+  @Override
+  public void addListener(MonitorUpdateListener listener) {
+    throw new IllegalStateException("Monitor is readOnly cannot register listeners");

Review comment:
       And here :)

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.*;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.NamedThreadFactory;
+
+class ReadonlyQueryIndex extends QueryIndex {
+
+  private final ScheduledExecutorService refreshExecutor;
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+    this.refreshExecutor =
+        Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("cache-purge"));
+    long refreshFrequency = configuration.getPurgeFrequency();
+    this.refreshExecutor.scheduleAtFixedRate(
+        () -> {
+          try {
+            manager.maybeRefresh();
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        },
+        refreshFrequency,
+        refreshFrequency,
+        configuration.getPurgeFrequencyUnits());
+  }
+
+  @Override
+  public void commit(List<MonitorQuery> updates) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot commit");
+  }
+
+  @Override
+  public long search(QueryBuilder queryBuilder, QueryCollector matcher) throws IOException {
+    IndexSearcher searcher = null;
+    try {
+      searcher = manager.acquire();
+      LazyMonitorQueryCollector collector =
+          new LazyMonitorQueryCollector(matcher, serializer, decomposer);
+      long buildTime = System.nanoTime();
+      Query query =
+          queryBuilder.buildQuery(
+              termFilters.get(searcher.getIndexReader().getReaderCacheHelper().getKey()));
+      buildTime = System.nanoTime() - buildTime;
+      searcher.search(query, collector);
+      return buildTime;
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+  }
+
+  @Override
+  public void purgeCache() {
+    throw new IllegalStateException("Monitor is readOnly, it has no cache");
+  }
+
+  @Override
+  void purgeCache(CachePopulator populator) {
+    throw new IllegalStateException("Monitor is readOnly, it has no cache");

Review comment:
       Can this throw `UnsupportedOperationException` instead?

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.*;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.NamedThreadFactory;
+
+class ReadonlyQueryIndex extends QueryIndex {
+
+  private final ScheduledExecutorService refreshExecutor;
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+    this.refreshExecutor =
+        Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("cache-purge"));
+    long refreshFrequency = configuration.getPurgeFrequency();
+    this.refreshExecutor.scheduleAtFixedRate(
+        () -> {
+          try {
+            manager.maybeRefresh();
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        },
+        refreshFrequency,
+        refreshFrequency,
+        configuration.getPurgeFrequencyUnits());
+  }
+
+  @Override
+  public void commit(List<MonitorQuery> updates) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot commit");
+  }
+
+  @Override
+  public long search(QueryBuilder queryBuilder, QueryCollector matcher) throws IOException {
+    IndexSearcher searcher = null;
+    try {
+      searcher = manager.acquire();
+      LazyMonitorQueryCollector collector =
+          new LazyMonitorQueryCollector(matcher, serializer, decomposer);
+      long buildTime = System.nanoTime();
+      Query query =
+          queryBuilder.buildQuery(
+              termFilters.get(searcher.getIndexReader().getReaderCacheHelper().getKey()));
+      buildTime = System.nanoTime() - buildTime;
+      searcher.search(query, collector);
+      return buildTime;
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+  }
+
+  @Override
+  public void purgeCache() {
+    throw new IllegalStateException("Monitor is readOnly, it has no cache");
+  }
+
+  @Override
+  void purgeCache(CachePopulator populator) {
+    throw new IllegalStateException("Monitor is readOnly, it has no cache");
+  }
+
+  @Override
+  public void close() throws IOException {
+    refreshExecutor.shutdown();
+    IOUtils.close(manager);
+  }
+
+  @Override
+  public int numDocs() throws IOException {
+    IndexSearcher searcher = null;
+    int numDocs;
+    try {
+      searcher = manager.acquire();
+      numDocs = searcher.getIndexReader().numDocs();
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+    return numDocs;
+  }
+
+  @Override
+  public int cacheSize() {
+    return -1;
+  }
+
+  @Override
+  public void deleteQueries(List<String> ids) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot delete queries");
+  }
+
+  @Override
+  public void clear() throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot clear");
+  }
+
+  @Override
+  public long getLastPurged() {
+    return -1;
+  }
+
+  @Override
+  public void addListener(MonitorUpdateListener listener) {
+    throw new IllegalStateException("Monitor is readOnly cannot register listeners");
+  }
+
+  // ---------------------------------------------
+  //  Helper classes...
+  // ---------------------------------------------
+
+  /** A Collector that decodes the stored query for each document hit reparsing them everytime. */
+  static final class LazyMonitorQueryCollector extends SimpleCollector {
+    private final QueryIndex.QueryCollector matcher;
+    private final QueryIndex.DataValues dataValues = new QueryIndex.DataValues();
+    private final MonitorQuerySerializer serializer;
+    private final QueryDecomposer decomposer;
+
+    LazyMonitorQueryCollector(
+        QueryIndex.QueryCollector matcher,
+        MonitorQuerySerializer serializer,
+        QueryDecomposer decomposer) {
+      this.matcher = matcher;
+      this.serializer = serializer;
+      this.decomposer = decomposer;
+    }
+
+    @Override
+    public void setScorer(Scorable scorer) {
+      this.dataValues.scorer = scorer;
+    }
+
+    @Override
+    public void collect(int doc) throws IOException {
+      dataValues.advanceTo(doc);
+      BytesRef cache_id = dataValues.cacheId.lookupOrd(dataValues.cacheId.ordValue());
+      BytesRef query_id = dataValues.queryId.lookupOrd(dataValues.queryId.ordValue());
+      MonitorQuery mq = serializer.deserialize(dataValues.mq.binaryValue());
+      QueryCacheEntry query =
+          QueryCacheEntry.decompose(mq, decomposer).stream()
+              .filter(queryCacheEntry -> queryCacheEntry.cacheId.equals(cache_id.utf8ToString()))
+              .findFirst()
+              .orElse(null);

Review comment:
       Can we throw an exception here instead? `null` is going to cause problems in the collector I think?

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.*;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.NamedThreadFactory;
+
+class ReadonlyQueryIndex extends QueryIndex {
+
+  private final ScheduledExecutorService refreshExecutor;
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+    this.refreshExecutor =
+        Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("cache-purge"));
+    long refreshFrequency = configuration.getPurgeFrequency();
+    this.refreshExecutor.scheduleAtFixedRate(
+        () -> {
+          try {
+            manager.maybeRefresh();
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        },
+        refreshFrequency,
+        refreshFrequency,
+        configuration.getPurgeFrequencyUnits());
+  }
+
+  @Override
+  public void commit(List<MonitorQuery> updates) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot commit");
+  }
+
+  @Override
+  public long search(QueryBuilder queryBuilder, QueryCollector matcher) throws IOException {
+    IndexSearcher searcher = null;
+    try {
+      searcher = manager.acquire();
+      LazyMonitorQueryCollector collector =
+          new LazyMonitorQueryCollector(matcher, serializer, decomposer);
+      long buildTime = System.nanoTime();
+      Query query =
+          queryBuilder.buildQuery(
+              termFilters.get(searcher.getIndexReader().getReaderCacheHelper().getKey()));
+      buildTime = System.nanoTime() - buildTime;
+      searcher.search(query, collector);
+      return buildTime;
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+  }
+
+  @Override
+  public void purgeCache() {
+    throw new IllegalStateException("Monitor is readOnly, it has no cache");
+  }
+
+  @Override
+  void purgeCache(CachePopulator populator) {
+    throw new IllegalStateException("Monitor is readOnly, it has no cache");
+  }
+
+  @Override
+  public void close() throws IOException {
+    refreshExecutor.shutdown();
+    IOUtils.close(manager);
+  }
+
+  @Override
+  public int numDocs() throws IOException {
+    IndexSearcher searcher = null;
+    int numDocs;
+    try {
+      searcher = manager.acquire();
+      numDocs = searcher.getIndexReader().numDocs();
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+    return numDocs;
+  }
+
+  @Override
+  public int cacheSize() {
+    return -1;
+  }
+
+  @Override
+  public void deleteQueries(List<String> ids) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot delete queries");
+  }
+
+  @Override
+  public void clear() throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot clear");

Review comment:
       And here

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.*;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.NamedThreadFactory;
+
+class ReadonlyQueryIndex extends QueryIndex {
+
+  private final ScheduledExecutorService refreshExecutor;
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+    this.refreshExecutor =
+        Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("cache-purge"));
+    long refreshFrequency = configuration.getPurgeFrequency();
+    this.refreshExecutor.scheduleAtFixedRate(
+        () -> {
+          try {
+            manager.maybeRefresh();
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        },
+        refreshFrequency,
+        refreshFrequency,
+        configuration.getPurgeFrequencyUnits());
+  }
+
+  @Override
+  public void commit(List<MonitorQuery> updates) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot commit");
+  }
+
+  @Override
+  public long search(QueryBuilder queryBuilder, QueryCollector matcher) throws IOException {
+    IndexSearcher searcher = null;
+    try {
+      searcher = manager.acquire();
+      LazyMonitorQueryCollector collector =
+          new LazyMonitorQueryCollector(matcher, serializer, decomposer);
+      long buildTime = System.nanoTime();
+      Query query =
+          queryBuilder.buildQuery(
+              termFilters.get(searcher.getIndexReader().getReaderCacheHelper().getKey()));
+      buildTime = System.nanoTime() - buildTime;
+      searcher.search(query, collector);
+      return buildTime;
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+  }
+
+  @Override
+  public void purgeCache() {
+    throw new IllegalStateException("Monitor is readOnly, it has no cache");
+  }
+
+  @Override
+  void purgeCache(CachePopulator populator) {
+    throw new IllegalStateException("Monitor is readOnly, it has no cache");
+  }
+
+  @Override
+  public void close() throws IOException {
+    refreshExecutor.shutdown();
+    IOUtils.close(manager);
+  }
+
+  @Override
+  public int numDocs() throws IOException {
+    IndexSearcher searcher = null;
+    int numDocs;
+    try {
+      searcher = manager.acquire();
+      numDocs = searcher.getIndexReader().numDocs();
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+    return numDocs;
+  }
+
+  @Override
+  public int cacheSize() {
+    return -1;
+  }
+
+  @Override
+  public void deleteQueries(List<String> ids) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot delete queries");

Review comment:
       Same here, this should be UnsupportedOperationException




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

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

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



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


[GitHub] [lucene] mogui commented on pull request #679: Monitor Improvements LUCENE-10422

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


   @romseygeek sorry to bother you, just a reminder :D


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

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

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



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


[GitHub] [lucene] mogui commented on pull request #679: Monitor Improvements LUCENE-10422

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


   Done!
   Thanks for Your patience and support @romseygeek !


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

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

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



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


[GitHub] [lucene] mogui commented on pull request #679: Monitor Improvements LUCENE-10422

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


   @romseygeek I've updated with the requested changes


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

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

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



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


[GitHub] [lucene] romseygeek commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.*;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex implements QueryIndex {
+  private final SearcherManager manager;
+  private final QueryDecomposer decomposer;
+  private final MonitorQuerySerializer serializer;
+
+  final Map<IndexReader.CacheKey, WritableQueryIndex.QueryTermFilter> termFilters = new HashMap<>();
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();

Review comment:
       So the way the Monitor works at the moment is that it parses all the serialized queries in its QueryIndex on startup, and stores them in an in-memory cache.  This means that when we run a document through the Monitor, once it has identified which candidate queries to run against it we don't need to re-parse them, they are already instantiated in RAM.  The alternative, which I think is what you've implemented here, is to re-parse the query every time we need to run it.  This is perfectly reasonable (in fact it's what elasticsearch's percolator does) but it is a significant change in behaviour so I don't think we should fold it in as part of this ticket.
   
   The Writeable query index needs to have all the tricksy behaviour around re-populating the cache, as we need to remove deleted entries or replace updated entries when queries are added or deleted (cache invalidation is hard, apparently!); the read-only index can just hold everything in a single Map that is populated on startup and never changes.




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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex extends QueryIndex {
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.queries = new HashMap<>();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+    this.populateQueryCache(serializer, decomposer);
+  }
+
+  @Override
+  public void commit(List<MonitorQuery> updates) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot commit");
+  }
+
+  @Override
+  long search(final Query query, QueryCollector matcher) throws IOException {
+    QueryBuilder builder = termFilter -> query;
+    return search(builder, matcher);
+  }
+
+  @Override
+  public long search(QueryBuilder queryBuilder, QueryCollector matcher) throws IOException {
+    IndexSearcher searcher = null;
+    try {
+      searcher = manager.acquire();
+      return searchInMemory(queryBuilder, matcher, searcher, this.queries);
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+  }
+
+  @Override
+  public void purgeCache() throws IOException {
+    this.populateQueryCache(serializer, decomposer);
+    lastPurged = System.nanoTime();
+  }
+
+  @Override
+  void purgeCache(CachePopulator populator) throws IOException {
+    manager.maybeRefresh();

Review comment:
       should I call `maybeRefreshBlocking` instead?




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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/QueryIndex.java
##########
@@ -87,8 +81,20 @@
   final Map<IndexReader.CacheKey, QueryTermFilter> termFilters = new HashMap<>();
 
   QueryIndex(MonitorConfiguration config, Presearcher presearcher) throws IOException {

Review comment:
       HI @romseygeek, thanks for the reply, ok for everything!
    I'll work on that, regarding the interface implementation split, that I agree it's more clear, How you suggest to let user select the implementattion? Keeping the readonly flag in MonitorConfiguration and instantiating the correct on in Monitor, should be enough?




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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/QueryIndex.java
##########
@@ -87,8 +81,20 @@
   final Map<IndexReader.CacheKey, QueryTermFilter> termFilters = new HashMap<>();
 
   QueryIndex(MonitorConfiguration config, Presearcher presearcher) throws IOException {

Review comment:
       I'm trying to wrap my head around it and tried few things, but from what I understand search methods cannot be isolted from cache population, so without a middle abstract class I'll end up with a lot of duplicate code.
   I don't like this Interface -> BaseAbstract and two implemtation hierarchy so much. We could skip the interface, but still I don't like so much.
   
   Maybe I'm missing something, but Monitor.search seems very coupled with loading in memory all stored queries in `ConcurrentMap<String, QueryCacheEntry> queries` so I have to replicate all query cache releated code.
   
   Am I wrong?




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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/test/org/apache/lucene/monitor/TestMonitorReadonly.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Collections;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.IndexNotFoundException;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.FSDirectory;
+import org.junit.Test;
+
+public class TestMonitorReadonly extends MonitorTestBase {
+  private static final Analyzer ANALYZER = new WhitespaceAnalyzer();
+
+  @Test
+  public void testReadonlyMonitorThrowsOnInexistentIndex() {
+    Path indexDirectory = createTempDir();
+    MonitorConfiguration config =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse),
+                true);
+    assertThrows(
+        IndexNotFoundException.class,
+        () -> {
+          new Monitor(ANALYZER, config);
+        });
+  }
+
+  @Test
+  public void testReadonlyMonitorThrowsWhenCallingWriteRequests() throws IOException {
+    Path indexDirectory = createTempDir();
+    MonitorConfiguration writeConfig =
+        new MonitorConfiguration()
+            .setIndexPath(
+                indexDirectory, MonitorQuerySerializer.fromParser(MonitorTestBase::parse));
+
+    // this will create the index
+    Monitor writeMonitor = new Monitor(ANALYZER, writeConfig);
+    writeMonitor.close();
+
+    MonitorConfiguration config =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse),
+                true);
+    try (Monitor monitor = new Monitor(ANALYZER, config)) {
+      assertThrows(
+          IllegalStateException.class,
+          () -> {
+            TermQuery query = new TermQuery(new Term(FIELD, "test"));
+            monitor.register(
+                new MonitorQuery("query1", query, query.toString(), Collections.emptyMap()));
+          });
+
+      assertThrows(
+          IllegalStateException.class,
+          () -> {
+            monitor.deleteById("query1");
+          });
+
+      assertThrows(
+          IllegalStateException.class,
+          () -> {
+            monitor.clear();
+          });
+    }
+  }
+
+  @Test
+  public void testSettingCustomDirectory() throws IOException {
+    Path indexDirectory = createTempDir();
+    Document doc = new Document();
+    doc.add(newTextField(FIELD, "This is a Foobar test document", Field.Store.NO));
+
+    MonitorConfiguration writeConfig =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse));
+
+    try (Monitor writeMonitor = new Monitor(ANALYZER, writeConfig)) {
+      TermQuery query = new TermQuery(new Term(FIELD, "test"));
+      writeMonitor.register(
+          new MonitorQuery("query1", query, query.toString(), Collections.emptyMap()));
+      TermQuery query2 = new TermQuery(new Term(FIELD, "Foobar"));
+      writeMonitor.register(
+          new MonitorQuery("query2", query2, query.toString(), Collections.emptyMap()));
+      MatchingQueries<QueryMatch> matches = writeMonitor.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertNotNull(matches.getMatches());
+      assertEquals(2, matches.getMatchCount());
+      assertNotNull(matches.matches("query2"));
+    }
+  }
+
+  public void testMonitorReadOnlyCouldReadOnTheSameIndex() throws IOException {
+    Path indexDirectory = createTempDir();
+    Document doc = new Document();
+    doc.add(newTextField(FIELD, "This is a test document", Field.Store.NO));
+
+    MonitorConfiguration writeConfig =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse));
+
+    try (Monitor writeMonitor = new Monitor(ANALYZER, writeConfig)) {
+      TermQuery query = new TermQuery(new Term(FIELD, "test"));
+      writeMonitor.register(
+          new MonitorQuery("query1", query, query.toString(), Collections.emptyMap()));
+    }
+
+    MonitorConfiguration readConfig =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse),
+                true);
+
+    try (Monitor readMonitor1 = new Monitor(ANALYZER, readConfig)) {
+      MatchingQueries<QueryMatch> matches = readMonitor1.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertNotNull(matches.getMatches());
+      assertEquals(1, matches.getMatchCount());
+      assertNotNull(matches.matches("query1"));
+    }
+
+    try (Monitor readMonitor2 = new Monitor(ANALYZER, readConfig)) {
+      MatchingQueries<QueryMatch> matches = readMonitor2.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertNotNull(matches.getMatches());
+      assertEquals(1, matches.getMatchCount());
+      assertNotNull(matches.matches("query1"));
+
+      assertThrows(
+          IllegalStateException.class,
+          () -> {
+            TermQuery query = new TermQuery(new Term(FIELD, "test"));
+            readMonitor2.register(
+                new MonitorQuery("query1", query, query.toString(), Collections.emptyMap()));
+          });
+    }
+  }

Review comment:
       Hi @romseygeek I am doing it, but indeed I cannot make it work, I am missing something?
   No change in monitor before purging it's ok, but after a purge still no changes, I wasn' calling maybeRefresh on the searchManager but also calling it before purging, I have still satale search results. What am I doing wrong ?




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

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

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



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


[GitHub] [lucene] mogui commented on pull request #679: Monitor Improvements LUCENE-10422

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


   @romseygeek I've merged main and restored listeners to the abstarct class, You were right the sleep was very ugly


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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/Monitor.java
##########
@@ -125,14 +108,21 @@ public Monitor(Analyzer analyzer, Presearcher presearcher, MonitorConfiguration
    * Monitor's queryindex
    *
    * @param listener listener to register
+   * @throws IllegalStateException when Monitor is readonly
    */
   public void addQueryIndexUpdateListener(MonitorUpdateListener listener) {
-    listeners.add(listener);

Review comment:
       yes 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.

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

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



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


[GitHub] [lucene] romseygeek commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex extends QueryIndex {
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+    this.populateQueryCache(serializer, decomposer);
+  }
+
+  @Override
+  public void commit(List<MonitorQuery> updates) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot commit");
+  }
+
+  @Override
+  long search(final Query query, QueryCollector matcher) throws IOException {
+    QueryBuilder builder = termFilter -> query;
+    return search(builder, matcher);
+  }
+
+  @Override
+  public long search(QueryBuilder queryBuilder, QueryCollector matcher) throws IOException {
+    IndexSearcher searcher = null;
+    try {
+      searcher = manager.acquire();
+      return searchInMemory(queryBuilder, matcher, searcher, this.queries);
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+  }
+
+  @Override
+  void purgeCache(CachePopulator populator) throws IOException {
+    final ConcurrentMap<String, QueryCacheEntry> newCache = new ConcurrentHashMap<>();

Review comment:
       I don't think this needs to be a concurrent hash map in this case, as we're not updating it elsewhere.

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex extends QueryIndex {
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+    this.populateQueryCache(serializer, decomposer);

Review comment:
       So this will still only repopulate the cache when you explicitly call `purgeCache()` on the parent Monitor I think?  That's probably OK but we should document that clearly.

##########
File path: lucene/monitor/src/test/org/apache/lucene/monitor/TestMonitorReadonly.java
##########
@@ -0,0 +1,165 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Collections;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.IndexNotFoundException;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.FSDirectory;
+import org.junit.Test;
+
+public class TestMonitorReadonly extends MonitorTestBase {
+  private static final Analyzer ANALYZER = new WhitespaceAnalyzer();
+
+  @Test
+  public void testReadonlyMonitorThrowsOnInexistentIndex() {
+    Path indexDirectory = createTempDir();
+    MonitorConfiguration config =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse),
+                true);
+    assertThrows(
+        IndexNotFoundException.class,
+        () -> {
+          new Monitor(ANALYZER, config);
+        });
+  }
+
+  @Test
+  public void testReadonlyMonitorThrowsWhenCallingWriteRequests() throws IOException {
+    Path indexDirectory = createTempDir();
+    MonitorConfiguration writeConfig =
+        new MonitorConfiguration()
+            .setIndexPath(
+                indexDirectory, MonitorQuerySerializer.fromParser(MonitorTestBase::parse));
+
+    // this will create the index
+    Monitor writeMonitor = new Monitor(ANALYZER, writeConfig);
+    writeMonitor.close();
+
+    MonitorConfiguration config =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse),
+                true);
+    try (Monitor monitor = new Monitor(ANALYZER, config)) {
+      assertThrows(
+          IllegalStateException.class,
+          () -> {
+            TermQuery query = new TermQuery(new Term(FIELD, "test"));
+            monitor.register(
+                new MonitorQuery("query1", query, query.toString(), Collections.emptyMap()));
+          });
+
+      assertThrows(
+          IllegalStateException.class,
+          () -> {
+            monitor.deleteById("query1");
+          });
+
+      assertThrows(
+          IllegalStateException.class,
+          () -> {
+            monitor.clear();
+          });
+    }
+  }
+
+  @Test
+  public void testSettingCustomDirectory() throws IOException {
+    Path indexDirectory = createTempDir();
+    Document doc = new Document();
+    doc.add(newTextField(FIELD, "This is a Foobar test document", Field.Store.NO));
+
+    MonitorConfiguration writeConfig =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse));
+
+    try (Monitor writeMonitor = new Monitor(ANALYZER, writeConfig)) {
+      TermQuery query = new TermQuery(new Term(FIELD, "test"));
+      writeMonitor.register(
+          new MonitorQuery("query1", query, query.toString(), Collections.emptyMap()));
+      TermQuery query2 = new TermQuery(new Term(FIELD, "Foobar"));
+      writeMonitor.register(
+          new MonitorQuery("query2", query2, query.toString(), Collections.emptyMap()));
+      MatchingQueries<QueryMatch> matches = writeMonitor.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertNotNull(matches.getMatches());
+      assertEquals(2, matches.getMatchCount());
+      assertNotNull(matches.matches("query2"));
+    }
+  }
+
+  public void testMonitorReadOnlyCouldReadOnTheSameIndex() throws IOException {
+    Path indexDirectory = createTempDir();
+    Document doc = new Document();
+    doc.add(newTextField(FIELD, "This is a test document", Field.Store.NO));
+
+    MonitorConfiguration writeConfig =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse));
+
+    try (Monitor writeMonitor = new Monitor(ANALYZER, writeConfig)) {
+      TermQuery query = new TermQuery(new Term(FIELD, "test"));
+      writeMonitor.register(
+          new MonitorQuery("query1", query, query.toString(), Collections.emptyMap()));
+    }
+
+    MonitorConfiguration readConfig =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse),
+                true);
+
+    try (Monitor readMonitor1 = new Monitor(ANALYZER, readConfig)) {
+      MatchingQueries<QueryMatch> matches = readMonitor1.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertNotNull(matches.getMatches());
+      assertEquals(1, matches.getMatchCount());
+      assertNotNull(matches.matches("query1"));
+    }
+
+    try (Monitor readMonitor2 = new Monitor(ANALYZER, readConfig)) {
+      MatchingQueries<QueryMatch> matches = readMonitor2.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertNotNull(matches.getMatches());
+      assertEquals(1, matches.getMatchCount());
+      assertNotNull(matches.matches("query1"));
+
+      assertThrows(
+          IllegalStateException.class,
+          () -> {
+            TermQuery query = new TermQuery(new Term(FIELD, "test"));
+            readMonitor2.register(
+                new MonitorQuery("query1", query, query.toString(), Collections.emptyMap()));
+          });
+    }
+  }

Review comment:
       Can you add a test for updating the write monitor? We should see no change in the read monitor until we call purgeCache.

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/WritableQueryIndex.java
##########
@@ -0,0 +1,253 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.lucene.document.*;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.NamedThreadFactory;
+
+class WritableQueryIndex extends QueryIndex {
+
+  private final IndexWriter writer;
+  private final Presearcher presearcher;
+
+  /* Used to cache updates while a purge is ongoing */
+  private volatile Map<String, QueryCacheEntry> purgeCache = null;
+
+  /* Used to lock around the creation of the purgeCache */
+  private final ReadWriteLock purgeLock = new ReentrantReadWriteLock();
+  private final Object commitLock = new Object();
+
+  private final ScheduledExecutorService purgeExecutor;
+  private final List<MonitorUpdateListener> listeners = new ArrayList<>();
+
+  WritableQueryIndex(MonitorConfiguration configuration, Presearcher presearcher)
+      throws IOException {
+
+    this.writer = configuration.buildIndexWriter();
+    this.manager = new SearcherManager(writer, true, true, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+    this.presearcher = presearcher;
+    populateQueryCache(serializer, decomposer);
+
+    long purgeFrequency = configuration.getPurgeFrequency();
+    this.purgeExecutor =
+        Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("cache-purge"));
+    this.purgeExecutor.scheduleAtFixedRate(
+        () -> {
+          try {
+            purgeCache();
+          } catch (Throwable e) {
+            listeners.forEach(l -> l.onPurgeError(e));
+          }
+        },
+        purgeFrequency,
+        purgeFrequency,
+        configuration.getPurgeFrequencyUnits());
+  }
+
+  @Override
+  public void addListener(MonitorUpdateListener listener) {
+    listeners.add(listener);
+  }
+
+  @Override
+  public void commit(List<MonitorQuery> updates) throws IOException {
+    commitWithoutNotify(updates);
+    listeners.forEach(l -> l.afterUpdate(updates));
+  }
+
+  private void commitWithoutNotify(List<MonitorQuery> updates) throws IOException {
+    List<Indexable> indexables = buildIndexables(updates);
+    synchronized (commitLock) {
+      purgeLock.readLock().lock();
+      try {
+        if (indexables.size() > 0) {
+          Set<String> ids = new HashSet<>();
+          for (Indexable update : indexables) {
+            ids.add(update.queryCacheEntry.queryId);
+          }
+          for (String id : ids) {
+            writer.deleteDocuments(new Term(FIELDS.query_id, id));
+          }
+          for (Indexable update : indexables) {
+            this.queries.put(update.queryCacheEntry.cacheId, update.queryCacheEntry);
+            writer.addDocument(update.document);
+            if (purgeCache != null)
+              purgeCache.put(update.queryCacheEntry.cacheId, update.queryCacheEntry);
+          }
+        }
+        writer.commit();
+        manager.maybeRefresh();
+      } finally {
+        purgeLock.readLock().unlock();
+      }
+    }
+  }
+
+  private static class Indexable {
+    final QueryCacheEntry queryCacheEntry;
+    final Document document;
+
+    private Indexable(QueryCacheEntry queryCacheEntry, Document document) {
+      this.queryCacheEntry = queryCacheEntry;
+      this.document = document;
+    }
+  }
+
+  private static final BytesRef EMPTY = new BytesRef();
+
+  private List<Indexable> buildIndexables(List<MonitorQuery> updates) {
+    List<Indexable> indexables = new ArrayList<>();
+    for (MonitorQuery mq : updates) {
+      if (serializer != null && mq.getQueryString() == null) {
+        throw new IllegalArgumentException(
+            "Cannot add a MonitorQuery with a null string representation to a non-ephemeral Monitor");
+      }
+      BytesRef serialized = serializer == null ? EMPTY : serializer.serialize(mq);
+      for (QueryCacheEntry qce : QueryCacheEntry.decompose(mq, decomposer)) {
+        Document doc = presearcher.indexQuery(qce.matchQuery, mq.getMetadata());
+        doc.add(new StringField(FIELDS.query_id, qce.queryId, Field.Store.NO));
+        doc.add(new SortedDocValuesField(FIELDS.cache_id, new BytesRef(qce.cacheId)));
+        doc.add(new SortedDocValuesField(FIELDS.query_id, new BytesRef(qce.queryId)));
+        doc.add(new BinaryDocValuesField(FIELDS.mq, serialized));
+        indexables.add(new Indexable(qce, doc));
+      }
+    }
+    return indexables;
+  }
+
+  @Override
+  public long search(QueryBuilder queryBuilder, QueryCollector matcher) throws IOException {
+    IndexSearcher searcher = null;
+    try {
+      Map<String, QueryCacheEntry> queries;
+
+      purgeLock.readLock().lock();
+      try {
+        searcher = manager.acquire();
+        queries = this.queries;
+      } finally {
+        purgeLock.readLock().unlock();
+      }
+
+      return searchInMemory(queryBuilder, matcher, searcher, queries);
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+  }
+
+  @Override
+  public void purgeCache() throws IOException {
+    super.purgeCache();
+    listeners.forEach(MonitorUpdateListener::onPurge);
+  }
+
+  @Override
+  /**
+   * Remove unused queries from the query cache.
+   *
+   * <p>This is normally called from a background thread at a rate set by configurePurgeFrequency().
+   *
+   * @throws IOException on IO errors
+   */
+  synchronized void purgeCache(CachePopulator populator) throws IOException {
+
+    // Note on implementation
+
+    // The purge works by scanning the query index and creating a new query cache populated
+    // for each query in the index.  When the scan is complete, the old query cache is swapped
+    // for the new, allowing it to be garbage-collected.
+
+    // In order to not drop cached queries that have been added while a purge is ongoing,
+    // we use a ReadWriteLock to guard the creation and removal of an register log.  Commits take
+    // the read lock.  If the register log has been created, then a purge is ongoing, and queries
+    // are added to the register log within the read lock guard.
+
+    // The purge takes the write lock when creating the register log, and then when swapping out
+    // the old query cache.  Within the second write lock guard, the contents of the register log
+    // are added to the new query cache, and the register log itself is removed.
+
+    final ConcurrentMap<String, QueryCacheEntry> newCache = new ConcurrentHashMap<>();
+
+    purgeLock.writeLock().lock();
+    try {
+      purgeCache = new ConcurrentHashMap<>();
+    } finally {
+      purgeLock.writeLock().unlock();
+    }
+
+    populator.populateCacheWithIndex(newCache);
+
+    purgeLock.writeLock().lock();
+    try {
+      newCache.putAll(purgeCache);
+      purgeCache = null;
+      queries = newCache;
+    } finally {
+      purgeLock.writeLock().unlock();
+    }
+  }
+
+  // ---------------------------------------------
+  //  Proxy trivial operations...
+  // ---------------------------------------------
+
+  @Override
+  public void close() throws IOException {
+    IOUtils.close(manager, writer, writer.getDirectory());
+    purgeExecutor.shutdown();

Review comment:
       Let's close the executor first to ensure we don't get spurious exceptions from a cache refresh operating on a closed directory.

##########
File path: lucene/monitor/src/test/org/apache/lucene/monitor/TestCachePurging.java
##########
@@ -155,7 +155,12 @@ public void testBackgroundPurges() throws IOException, InterruptedException {
             @Override
             public void onPurge() {
               // It can sometimes take a couple of purge runs to get everything in sync
-              if (monitor.getQueryCacheStats().cachedQueries == 99) latch.countDown();
+              try {
+                if (monitor.getQueryCacheStats().cachedQueries == 99) latch.countDown();
+              } catch (IOException e) {
+                // Ignore
+                System.out.println(e);

Review comment:
       Instead of a println can you rethrow this as a RuntimeException?




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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex extends QueryIndex {
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.queries = new HashMap<>();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+    this.populateQueryCache(serializer, decomposer);
+  }
+
+  @Override
+  public void commit(List<MonitorQuery> updates) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot commit");
+  }
+
+  @Override
+  long search(final Query query, QueryCollector matcher) throws IOException {
+    QueryBuilder builder = termFilter -> query;
+    return search(builder, matcher);
+  }
+
+  @Override
+  public long search(QueryBuilder queryBuilder, QueryCollector matcher) throws IOException {
+    IndexSearcher searcher = null;
+    try {
+      searcher = manager.acquire();
+      return searchInMemory(queryBuilder, matcher, searcher, this.queries);
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+  }
+
+  @Override
+  public void purgeCache() throws IOException {
+    this.populateQueryCache(serializer, decomposer);
+    lastPurged = System.nanoTime();
+  }
+
+  @Override
+  void purgeCache(CachePopulator populator) throws IOException {
+    manager.maybeRefresh();

Review comment:
       @romseygeek in your opinion it makes sense to reuse `purgeFrequency` and `purgeFrequencyUnits`  from `MonitorConfiguration` to configure the refreshing thread or is better a new pairof configurations?




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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,117 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex extends QueryIndex {
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.queries = new HashMap<>();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+    this.populateQueryCache(serializer, decomposer);
+  }
+
+  @Override
+  public void commit(List<MonitorQuery> updates) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot commit");
+  }
+
+  @Override
+  long search(final Query query, QueryCollector matcher) throws IOException {
+    QueryBuilder builder = termFilter -> query;
+    return search(builder, matcher);
+  }
+
+  @Override
+  public long search(QueryBuilder queryBuilder, QueryCollector matcher) throws IOException {
+    IndexSearcher searcher = null;
+    try {
+      searcher = manager.acquire();
+      return searchInMemory(queryBuilder, matcher, searcher, this.queries);
+    } finally {
+      if (searcher != null) {
+        manager.release(searcher);
+      }
+    }
+  }
+
+  @Override
+  public void purgeCache() throws IOException {
+    this.populateQueryCache(serializer, decomposer);
+    lastPurged = System.nanoTime();
+  }
+
+  @Override
+  void purgeCache(CachePopulator populator) throws IOException {
+    manager.maybeRefresh();

Review comment:
       Ok, I think it was the best solution too, I'll work getting back to that solution.
   Don't worry, all the back and forth got me to understand everything 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.

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

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



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


[GitHub] [lucene] mogui commented on pull request #679: Monitor Improvements LUCENE-10422

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


   @romseygeek fixed !


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

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

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



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


[GitHub] [lucene] mogui commented on pull request #679: Monitor Improvements LUCENE-10422

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


   @romseygeek There is something else  in roadmap or future ideas regarding this package that I can contribute? Or where I can look to contribute more to the project. I'd like to remain involved if I can, it was nice to wotk with. 


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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/QueryIndex.java
##########
@@ -87,8 +81,20 @@
   final Map<IndexReader.CacheKey, QueryTermFilter> termFilters = new HashMap<>();
 
   QueryIndex(MonitorConfiguration config, Presearcher presearcher) throws IOException {

Review comment:
       Yes, it makes sense.




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

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

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



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


[GitHub] [lucene] romseygeek commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/QueryIndex.java
##########
@@ -87,8 +81,20 @@
   final Map<IndexReader.CacheKey, QueryTermFilter> termFilters = new HashMap<>();
 
   QueryIndex(MonitorConfiguration config, Presearcher presearcher) throws IOException {

Review comment:
       > Keeping the readonly flag in MonitorConfiguration and instantiating the correct on in Monitor, should be enough?
   
   I think so.  Maybe add it as a second parameter to `setDirectoryProvider` as it only really makes sense in combination with a custom Directory?




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

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

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



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


[GitHub] [lucene] romseygeek commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/MonitorConfiguration.java
##########
@@ -35,8 +34,9 @@
   private long purgeFrequency = 5;
   private TimeUnit purgeFrequencyUnits = TimeUnit.MINUTES;
   private QueryDecomposer queryDecomposer = new QueryDecomposer();
-  private Path indexPath = null;
   private MonitorQuerySerializer serializer;
+  private Boolean readOnly = false;
+  private DirectoryProviderFunctionalInterface directoryProvider;

Review comment:
       We can set this to a default value of `() -> new ByteBuffersDirectory()` which would then simplify the `buildIndexWriter()` implementation.

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/DirectoryProviderFunctionalInterface.java
##########
@@ -0,0 +1,32 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import org.apache.lucene.store.Directory;
+
+/** A functional interface to provide a Directory instance */
+public interface DirectoryProviderFunctionalInterface {

Review comment:
       Can we use `org.apache.lucene.util.IOSupplier` here instead please?

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/QueryIndex.java
##########
@@ -87,8 +81,20 @@
   final Map<IndexReader.CacheKey, QueryTermFilter> termFilters = new HashMap<>();
 
   QueryIndex(MonitorConfiguration config, Presearcher presearcher) throws IOException {

Review comment:
       Given that QueryIndex is package-private I think we have some leeway to split things up here a bit and make things clearer.  Could you try making QueryIndex itself an interface, and then having read-only and writeable implementations?  We could also move the purge executor from the Monitor into the writeable QueryIndex implementation as it's not necessary to clear the cache at all on a read-only query index.

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/QueryIndex.java
##########
@@ -36,15 +36,7 @@
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.SortedDocValuesField;
 import org.apache.lucene.document.StringField;
-import org.apache.lucene.index.BinaryDocValues;

Review comment:
       The code formatter will complain about these imports I think

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/QueryIndex.java
##########
@@ -87,8 +81,20 @@
   final Map<IndexReader.CacheKey, QueryTermFilter> termFilters = new HashMap<>();
 
   QueryIndex(MonitorConfiguration config, Presearcher presearcher) throws IOException {

Review comment:
       > Keeping the readonly flag in MonitorConfiguration and instantiating the correct on in Monitor, should be enough?
   
   I think so.  Maybe add it as a second parameter to `setDirectoryProvider` as it only really makes sense in combination with a custom Directory?




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

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

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



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


[GitHub] [lucene] romseygeek commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.*;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex implements QueryIndex {
+  private final SearcherManager manager;
+  private final QueryDecomposer decomposer;
+  private final MonitorQuerySerializer serializer;
+
+  final Map<IndexReader.CacheKey, WritableQueryIndex.QueryTermFilter> termFilters = new HashMap<>();
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();

Review comment:
       AIUI, this implementation doesn't have an in-memory query cache, and re-parses the queries every time we do a match.  I think having a lazy parser is definitely a valid use-case but I think we should decouple it from the notion of a read-only monitor.

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/Monitor.java
##########
@@ -125,14 +108,21 @@ public Monitor(Analyzer analyzer, Presearcher presearcher, MonitorConfiguration
    * Monitor's queryindex
    *
    * @param listener listener to register
+   * @throws IllegalStateException when Monitor is readonly
    */
   public void addQueryIndexUpdateListener(MonitorUpdateListener listener) {
-    listeners.add(listener);

Review comment:
       I think we can just make `addListener()` a method on `QueryIndex` and delegate there? And then we don't need the `readOnly` member variable on `Monitor`

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.*;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex implements QueryIndex {
+  private final SearcherManager manager;
+  private final QueryDecomposer decomposer;
+  private final MonitorQuerySerializer serializer;
+
+  final Map<IndexReader.CacheKey, WritableQueryIndex.QueryTermFilter> termFilters = new HashMap<>();
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+  }
+
+  @Override
+  public void commit(List<MonitorQuery> updates) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot commit");
+  }
+
+  @Override
+  public MonitorQuery getQuery(String queryId) throws IOException {
+    if (serializer == null) {
+      throw new IllegalStateException(
+          "Cannot get queries from an index with no MonitorQuerySerializer");
+    }
+    BytesRef[] bytesHolder = new BytesRef[1];
+    search(
+        new TermQuery(new Term(WritableQueryIndex.FIELDS.query_id, queryId)),

Review comment:
       Given that we're re-using a fair amount of stuff from WriteableQueryIndex here, it probably makes more sense to make QueryIndex an abstract class and move the shared code there.

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/MonitorConfiguration.java
##########
@@ -47,16 +49,39 @@ private static IndexWriterConfig defaultIndexWriterConfig() {
     return iwc;
   }
 
+  public Boolean isReadOnly() {

Review comment:
       Can this be a plain `boolean`?




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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.*;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex implements QueryIndex {
+  private final SearcherManager manager;
+  private final QueryDecomposer decomposer;
+  private final MonitorQuerySerializer serializer;
+
+  final Map<IndexReader.CacheKey, WritableQueryIndex.QueryTermFilter> termFilters = new HashMap<>();
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();

Review comment:
       Yes, it all goes to the `MonitorQueryCollector` that relies on the in-memory query  cache, and it is an internal class of WritableQueryIndex.
   
   Are you suggesting to decouple `ReadonlyMonitorQueryCollector` as a lazy query parser,  outside Readonly Monitor ?
   Am I getting it right? @romseygeek 




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

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

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



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


[GitHub] [lucene] romseygeek commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/Monitor.java
##########
@@ -125,14 +105,16 @@ public Monitor(Analyzer analyzer, Presearcher presearcher, MonitorConfiguration
    * Monitor's queryindex
    *
    * @param listener listener to register
+   * @throws IllegalStateException when Monitor is readonly

Review comment:
       I think this is an UOE now? Probably doesn't need to be in the javadoc, to be honest.

##########
File path: lucene/monitor/src/test/org/apache/lucene/monitor/TestMonitorReadonly.java
##########
@@ -0,0 +1,217 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.nio.file.Path;
+import java.util.Collections;
+import java.util.concurrent.TimeUnit;
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.IndexNotFoundException;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.FSDirectory;
+import org.junit.Test;
+
+public class TestMonitorReadonly extends MonitorTestBase {
+  private static final Analyzer ANALYZER = new WhitespaceAnalyzer();
+
+  @Test
+  public void testReadonlyMonitorThrowsOnInexistentIndex() {
+    Path indexDirectory = createTempDir();
+    MonitorConfiguration config =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse),
+                true);
+    assertThrows(
+        IndexNotFoundException.class,
+        () -> {
+          new Monitor(ANALYZER, config);
+        });
+  }
+
+  @Test
+  public void testReadonlyMonitorThrowsWhenCallingWriteRequests() throws IOException {
+    Path indexDirectory = createTempDir();
+    MonitorConfiguration writeConfig =
+        new MonitorConfiguration()
+            .setIndexPath(
+                indexDirectory, MonitorQuerySerializer.fromParser(MonitorTestBase::parse));
+
+    // this will create the index
+    Monitor writeMonitor = new Monitor(ANALYZER, writeConfig);
+    writeMonitor.close();
+
+    MonitorConfiguration config =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse),
+                true);
+    try (Monitor monitor = new Monitor(ANALYZER, config)) {
+      assertThrows(
+          IllegalStateException.class,
+          () -> {
+            TermQuery query = new TermQuery(new Term(FIELD, "test"));
+            monitor.register(
+                new MonitorQuery("query1", query, query.toString(), Collections.emptyMap()));
+          });
+
+      assertThrows(
+          UnsupportedOperationException.class,
+          () -> {
+            monitor.deleteById("query1");
+          });
+
+      assertThrows(
+          UnsupportedOperationException.class,
+          () -> {
+            monitor.clear();
+          });
+    }
+  }
+
+  @Test
+  public void testSettingCustomDirectory() throws IOException {
+    Path indexDirectory = createTempDir();
+    Document doc = new Document();
+    doc.add(newTextField(FIELD, "This is a Foobar test document", Field.Store.NO));
+
+    MonitorConfiguration writeConfig =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse));
+
+    try (Monitor writeMonitor = new Monitor(ANALYZER, writeConfig)) {
+      TermQuery query = new TermQuery(new Term(FIELD, "test"));
+      writeMonitor.register(
+          new MonitorQuery("query1", query, query.toString(), Collections.emptyMap()));
+      TermQuery query2 = new TermQuery(new Term(FIELD, "Foobar"));
+      writeMonitor.register(
+          new MonitorQuery("query2", query2, query.toString(), Collections.emptyMap()));
+      MatchingQueries<QueryMatch> matches = writeMonitor.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertNotNull(matches.getMatches());
+      assertEquals(2, matches.getMatchCount());
+      assertNotNull(matches.matches("query2"));
+    }
+  }
+
+  @Test
+  public void testMonitorReadOnlyCouldReadOnTheSameIndex() throws IOException {
+    Path indexDirectory = createTempDir();
+    Document doc = new Document();
+    doc.add(newTextField(FIELD, "This is a test document", Field.Store.NO));
+
+    MonitorConfiguration writeConfig =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse));
+
+    try (Monitor writeMonitor = new Monitor(ANALYZER, writeConfig)) {
+      TermQuery query = new TermQuery(new Term(FIELD, "test"));
+      writeMonitor.register(
+          new MonitorQuery("query1", query, query.toString(), Collections.emptyMap()));
+    }
+
+    MonitorConfiguration readConfig =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse),
+                true);
+
+    try (Monitor readMonitor1 = new Monitor(ANALYZER, readConfig)) {
+      MatchingQueries<QueryMatch> matches = readMonitor1.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertNotNull(matches.getMatches());
+      assertEquals(1, matches.getMatchCount());
+      assertNotNull(matches.matches("query1"));
+    }
+
+    try (Monitor readMonitor2 = new Monitor(ANALYZER, readConfig)) {
+      MatchingQueries<QueryMatch> matches = readMonitor2.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertNotNull(matches.getMatches());
+      assertEquals(1, matches.getMatchCount());
+      assertNotNull(matches.matches("query1"));
+
+      assertThrows(
+          IllegalStateException.class,
+          () -> {
+            TermQuery query = new TermQuery(new Term(FIELD, "test"));
+            readMonitor2.register(
+                new MonitorQuery("query1", query, query.toString(), Collections.emptyMap()));
+          });
+    }
+  }
+
+  @Test
+  public void testReadonlyMonitorGetsRefreshed() throws IOException, InterruptedException {
+    Path indexDirectory = createTempDir();
+    Document doc = new Document();
+    doc.add(newTextField(FIELD, "This is a test document", Field.Store.NO));
+
+    MonitorConfiguration writeConfig =
+        new MonitorConfiguration()
+            .setDirectoryProvider(
+                () -> FSDirectory.open(indexDirectory),
+                MonitorQuerySerializer.fromParser(MonitorTestBase::parse));
+
+    try (Monitor writeMonitor = new Monitor(ANALYZER, writeConfig)) {
+      TermQuery query = new TermQuery(new Term(FIELD, "test"));
+      writeMonitor.register(
+          new MonitorQuery("query1", query, query.toString(), Collections.emptyMap()));
+
+      MonitorConfiguration readConfig =
+          new MonitorConfiguration()
+              .setPurgeFrequency(2, TimeUnit.SECONDS)
+              .setDirectoryProvider(
+                  () -> FSDirectory.open(indexDirectory),
+                  MonitorQuerySerializer.fromParser(MonitorTestBase::parse),
+                  true);
+
+      try (Monitor readMonitor = new Monitor(ANALYZER, readConfig)) {
+        MatchingQueries<QueryMatch> matches = readMonitor.match(doc, QueryMatch.SIMPLE_MATCHER);
+        assertNotNull(matches.getMatches());
+        assertEquals(1, matches.getMatchCount());
+        assertNotNull(matches.matches("query1"));
+
+        TermQuery query2 = new TermQuery(new Term(FIELD, "test"));
+        writeMonitor.register(
+            new MonitorQuery("query2", query2, query2.toString(), Collections.emptyMap()));
+
+        // Index returns stale result until background refresh thread calls maybeRefresh
+        MatchingQueries<QueryMatch> matches2 = readMonitor.match(doc, QueryMatch.SIMPLE_MATCHER);
+        assertNotNull(matches2.getMatches());
+        assertEquals(1, matches2.getMatchCount());
+
+        TimeUnit.SECONDS.sleep(readConfig.getPurgeFrequency() + 1);

Review comment:
       I'd like to avoid sleeps in tests if we can.  Maybe add back the ability to register a MonitorUpdateListener on a read-only monitor, and use a CountDownLatch that is triggered when onPurge() is called in a listener?
   
   Something like:
   ```
   CountDownLatch latch = new CountDownLatch(1);
   readMonitor.addQueryIndexUpdateListener(new MonitorUpdateListener(){
     @Override
     void onPurge() {
       latch.countDown();
     }
   };
   latch.await();
   ```

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,194 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.*;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.NamedThreadFactory;
+
+class ReadonlyQueryIndex extends QueryIndex {
+
+  private final ScheduledExecutorService refreshExecutor;
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+    this.refreshExecutor =
+        Executors.newSingleThreadScheduledExecutor(new NamedThreadFactory("cache-purge"));
+    long refreshFrequency = configuration.getPurgeFrequency();
+    this.refreshExecutor.scheduleAtFixedRate(
+        () -> {
+          try {
+            manager.maybeRefresh();
+          } catch (IOException e) {
+            throw new RuntimeException(e);
+          }
+        },
+        refreshFrequency,
+        refreshFrequency,
+        configuration.getPurgeFrequencyUnits());
+  }
+
+  @Override
+  public void commit(List<MonitorQuery> updates) throws IOException {
+    throw new IllegalStateException("Monitor is readOnly cannot commit");

Review comment:
       UnsupportedOperationException




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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/QueryIndex.java
##########
@@ -87,8 +81,20 @@
   final Map<IndexReader.CacheKey, QueryTermFilter> termFilters = new HashMap<>();
 
   QueryIndex(MonitorConfiguration config, Presearcher presearcher) throws IOException {

Review comment:
       I'm trying to wrap my head around it and tried few things, but from what I understand search methods cannot be isolated from cache population, so without a middle abstract class I'll end up with a lot of duplicate code.
   I don't like to have  `Interface -> BaseAbstract` and two implemtation hierarchy so much. We could skip the interface, but still I don't like it.
   
   Maybe I'm missing something, but Monitor.search seems very coupled with loading in memory all stored queries in `ConcurrentMap<String, QueryCacheEntry> queries` so I have to replicate all query cache releated code.
   
   Am I wrong?




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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/QueryIndex.java
##########
@@ -87,8 +81,20 @@
   final Map<IndexReader.CacheKey, QueryTermFilter> termFilters = new HashMap<>();
 
   QueryIndex(MonitorConfiguration config, Presearcher presearcher) throws IOException {

Review comment:
       HI @romseygeek, thanks for the reply, ok for everything!
    I'll work on that, regarding the interface implementation split, that I agree it's more clear, How you suggest to let user select the implementattion? Keeping the readonly flag in MonitorConfiguration and instantiating the correct on in Monitor, should be enough?

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/QueryIndex.java
##########
@@ -87,8 +81,20 @@
   final Map<IndexReader.CacheKey, QueryTermFilter> termFilters = new HashMap<>();
 
   QueryIndex(MonitorConfiguration config, Presearcher presearcher) throws IOException {

Review comment:
       Yes, it makes sense.

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/QueryIndex.java
##########
@@ -87,8 +81,20 @@
   final Map<IndexReader.CacheKey, QueryTermFilter> termFilters = new HashMap<>();
 
   QueryIndex(MonitorConfiguration config, Presearcher presearcher) throws IOException {

Review comment:
       I'm trying to wrap my head around it and tried few things, but from what I understand search methods cannot be isolted from cache population, so without a middle abstract class I'll end up with a lot of duplicate code.
   I don't like this Interface -> BaseAbstract and two implemtation hierarchy so much. We could skip the interface, but still I don't like so much.
   
   Maybe I'm missing something, but Monitor.search seems very coupled with loading in memory all stored queries in `ConcurrentMap<String, QueryCacheEntry> queries` so I have to replicate all query cache releated code.
   
   Am I wrong?

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/QueryIndex.java
##########
@@ -87,8 +81,20 @@
   final Map<IndexReader.CacheKey, QueryTermFilter> termFilters = new HashMap<>();
 
   QueryIndex(MonitorConfiguration config, Presearcher presearcher) throws IOException {

Review comment:
       I'm trying to wrap my head around it and tried few things, but from what I understand search methods cannot be isolated from cache population, so without a middle abstract class I'll end up with a lot of duplicate code.
   I don't like to have  `Interface -> BaseAbstract` and two implemtation hierarchy so much. We could skip the interface, but still I don't like it.
   
   Maybe I'm missing something, but Monitor.search seems very coupled with loading in memory all stored queries in `ConcurrentMap<String, QueryCacheEntry> queries` so I have to replicate all query cache releated code.
   
   Am I wrong?

##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/QueryIndex.java
##########
@@ -87,8 +81,20 @@
   final Map<IndexReader.CacheKey, QueryTermFilter> termFilters = new HashMap<>();
 
   QueryIndex(MonitorConfiguration config, Presearcher presearcher) throws IOException {

Review comment:
       Nevermind, I got it or maybe I just think i did :)
   
   Tomorrow I'll update the pull request.




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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.*;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex implements QueryIndex {
+  private final SearcherManager manager;
+  private final QueryDecomposer decomposer;
+  private final MonitorQuerySerializer serializer;
+
+  final Map<IndexReader.CacheKey, WritableQueryIndex.QueryTermFilter> termFilters = new HashMap<>();
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();

Review comment:
       What do you think if I keep all the in-memory cache things along with the purgeExecutor in the abstract QueryIndex class and let the readonly monitor use it too, closing this round of improvements.
   Then open another changeset to implement the lazy parsing.
   




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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.*;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex implements QueryIndex {
+  private final SearcherManager manager;
+  private final QueryDecomposer decomposer;
+  private final MonitorQuerySerializer serializer;
+
+  final Map<IndexReader.CacheKey, WritableQueryIndex.QueryTermFilter> termFilters = new HashMap<>();
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();

Review comment:
       Ah, good to know! What do you think it's best to do? Makes sense what I have proposed?




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

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

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



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


[GitHub] [lucene] mogui edited a comment on pull request #679: Monitor Improvements LUCENE-10422

Posted by GitBox <gi...@apache.org>.
mogui edited a comment on pull request #679:
URL: https://github.com/apache/lucene/pull/679#issuecomment-1044239703


   @romseygeek OK I've moved things around, making QueryIndex an abstract class sharing common code for both implementations
   ReadonlyQueryIndex have the same logic of in-memory laoding of parsed queries, but without locking handling and without purge executor.
   I've made it usable by enabling purging cache so it can be done externally.
   
   


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

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

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



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


[GitHub] [lucene] mogui commented on pull request #679: Monitor Improvements LUCENE-10422

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


   OK I've moved things around, making QueryIndex an abstract class sharing common code for both implementations
   ReadonlyQueryIndex have the same logic of in-memory laoding of parsed queries, but without locking handling and without purge executor.
   I've made it usable by enabling purging cache so it can be done externally.
   
   


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

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

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



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


[GitHub] [lucene] mogui commented on pull request #679: Monitor Improvements LUCENE-10422

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


   Ok!


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

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

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



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


[GitHub] [lucene] romseygeek commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/CHANGES.txt
##########
@@ -56,7 +56,7 @@ API Changes
 
 New Features
 ---------------------
-(No changes)
+* LUCENE-10422: Monitor Improvements: `Monitor` can use a custom `Directory` implementation. `Monitor` can be created with a readonly `QueryIndex` in order to have readonly `Monitor` instances. (Niko Usai)

Review comment:
       ```suggestion
   * LUCENE-10422: Monitor Improvements: `Monitor` can use a custom `Directory` 
   implementation. `Monitor` can be created with a readonly `QueryIndex` in order to 
   have readonly `Monitor` instances. (Niko Usai)
   ```




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

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

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



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


[GitHub] [lucene] mogui commented on a change in pull request #679: Monitor Improvements LUCENE-10422

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



##########
File path: lucene/monitor/src/java/org/apache/lucene/monitor/ReadonlyQueryIndex.java
##########
@@ -0,0 +1,100 @@
+/*
+ * 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.monitor;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+
+class ReadonlyQueryIndex extends QueryIndex {
+
+  public ReadonlyQueryIndex(MonitorConfiguration configuration) throws IOException {
+    if (configuration.getDirectoryProvider() == null) {
+      throw new IllegalStateException(
+          "You must specify a Directory when configuring a Monitor as read-only.");
+    }
+    Directory directory = configuration.getDirectoryProvider().get();
+    this.manager = new SearcherManager(directory, new TermsHashBuilder(termFilters));
+    this.decomposer = configuration.getQueryDecomposer();
+    this.serializer = configuration.getQuerySerializer();
+    this.populateQueryCache(serializer, decomposer);

Review comment:
       Yes.
   @romseygeek Do you think it could make sense using the purge executor here too?




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

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

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



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