You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2019/06/13 09:15:24 UTC

[lucene-solr] branch master updated: LUCENE-8766: Add monitor subproject

This is an automated email from the ASF dual-hosted git repository.

romseygeek pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new 251dbe7  LUCENE-8766: Add monitor subproject
251dbe7 is described below

commit 251dbe7cea8126cb7ecadc921ba444c63d06c618
Author: Alan Woodward <ro...@apache.org>
AuthorDate: Thu Jun 13 09:40:57 2019 +0100

    LUCENE-8766: Add monitor subproject
---
 dev-tools/idea/.idea/ant.xml                       |   1 +
 dev-tools/idea/.idea/modules.xml                   |   1 +
 dev-tools/idea/.idea/workspace.xml                 |   8 +
 dev-tools/idea/lucene/monitor/monitor.iml          |  32 ++
 dev-tools/maven/lucene/monitor/pom.xml.template    |  70 +++
 dev-tools/maven/lucene/pom.xml.template            |   1 +
 lucene/monitor/build.xml                           |  53 +++
 lucene/monitor/ivy.xml                             |  21 +
 .../apache/lucene/monitor/CandidateMatcher.java    | 133 ++++++
 .../apache/lucene/monitor/CollectingMatcher.java   |  84 ++++
 .../lucene/monitor/ConcurrentQueryLoader.java      | 196 ++++++++
 .../apache/lucene/monitor/CustomQueryHandler.java  |  45 ++
 .../org/apache/lucene/monitor/DocumentBatch.java   | 121 +++++
 .../org/apache/lucene/monitor/ExplainingMatch.java |  81 ++++
 .../lucene/monitor/ForceNoBulkScoringQuery.java    | 103 ++++
 .../org/apache/lucene/monitor/HighlightsMatch.java | 230 +++++++++
 .../org/apache/lucene/monitor/MatcherFactory.java  |  35 ++
 .../org/apache/lucene/monitor/MatchingQueries.java |  97 ++++
 .../java/org/apache/lucene/monitor/Monitor.java    | 409 ++++++++++++++++
 .../lucene/monitor/MonitorConfiguration.java       | 135 ++++++
 .../org/apache/lucene/monitor/MonitorQuery.java    | 135 ++++++
 .../lucene/monitor/MonitorQuerySerializer.java     |  92 ++++
 .../lucene/monitor/MonitorUpdateListener.java      |  52 ++
 .../lucene/monitor/MultiMatchingQueries.java       | 123 +++++
 .../monitor/MultipassTermFilteredPresearcher.java  | 161 +++++++
 .../org/apache/lucene/monitor/ParallelMatcher.java | 205 ++++++++
 .../apache/lucene/monitor/PartitionMatcher.java    | 202 ++++++++
 .../org/apache/lucene/monitor/Presearcher.java     |  71 +++
 .../apache/lucene/monitor/PresearcherMatch.java    |  45 ++
 .../apache/lucene/monitor/PresearcherMatches.java  |  50 ++
 .../org/apache/lucene/monitor/QueryAnalyzer.java   | 143 ++++++
 .../org/apache/lucene/monitor/QueryCacheEntry.java |  71 +++
 .../org/apache/lucene/monitor/QueryDecomposer.java | 123 +++++
 .../java/org/apache/lucene/monitor/QueryIndex.java | 449 ++++++++++++++++++
 .../java/org/apache/lucene/monitor/QueryMatch.java |  84 ++++
 .../apache/lucene/monitor/QueryTimeListener.java   |  64 +++
 .../java/org/apache/lucene/monitor/QueryTree.java  | 299 ++++++++++++
 .../apache/lucene/monitor/RegexpQueryHandler.java  | 155 ++++++
 .../org/apache/lucene/monitor/ScoringMatch.java    |  81 ++++
 .../java/org/apache/lucene/monitor/SlowLog.java    |  88 ++++
 .../lucene/monitor/SuffixingNGramTokenFilter.java  | 144 ++++++
 .../lucene/monitor/TermFilteredPresearcher.java    | 309 ++++++++++++
 .../org/apache/lucene/monitor/TermWeightor.java    | 152 ++++++
 .../lucene/monitor/TermsEnumTokenStream.java       |  54 +++
 .../org/apache/lucene/monitor/package-info.java    |  98 ++++
 lucene/monitor/src/java/overview.html              |  26 +
 .../lucene/monitor/ConcurrentMatcherTestBase.java  |  96 ++++
 .../FieldFilterPresearcherComponentTestBase.java   | 141 ++++++
 .../org/apache/lucene/monitor/MonitorTestBase.java |  93 ++++
 .../apache/lucene/monitor/PresearcherTestBase.java | 178 +++++++
 .../monitor/TestBooleanClauseWeightings.java       |  64 +++
 .../lucene/monitor/TestBooleanTermExtractor.java   | 135 ++++++
 .../apache/lucene/monitor/TestCachePurging.java    | 153 ++++++
 .../lucene/monitor/TestConcurrentQueryLoader.java  |  33 ++
 .../lucene/monitor/TestExplainingMatcher.java      |  53 +++
 .../org/apache/lucene/monitor/TestExtractors.java  | 103 ++++
 .../TestFieldFilteredMultipassPresearcher.java     |  29 ++
 .../monitor/TestFieldTermFilteredPresearcher.java  |  28 ++
 .../monitor/TestForceNoBulkScoringQuery.java       |  82 ++++
 .../lucene/monitor/TestHighlightingMatcher.java    | 527 +++++++++++++++++++++
 .../lucene/monitor/TestMatchAllPresearcher.java    |  27 ++
 .../org/apache/lucene/monitor/TestMonitor.java     | 245 ++++++++++
 .../lucene/monitor/TestMonitorErrorHandling.java   |  57 +++
 .../lucene/monitor/TestMonitorPersistence.java     |  72 +++
 .../lucene/monitor/TestMultipassPresearcher.java   | 123 +++++
 .../apache/lucene/monitor/TestParallelMatcher.java |  28 ++
 .../lucene/monitor/TestPartitionMatcher.java       |  60 +++
 .../monitor/TestPresearcherMatchCollector.java     |  61 +++
 .../apache/lucene/monitor/TestQueryAnalyzer.java   | 194 ++++++++
 .../apache/lucene/monitor/TestQueryDecomposer.java | 110 +++++
 .../lucene/monitor/TestQueryTermComparators.java   | 103 ++++
 .../apache/lucene/monitor/TestQueryTermFilter.java |  52 ++
 .../lucene/monitor/TestRegexpQueryHandler.java     |  84 ++++
 .../apache/lucene/monitor/TestSimilarities.java    |  50 ++
 .../apache/lucene/monitor/TestSimpleMatcher.java   |  40 ++
 .../apache/lucene/monitor/TestSpanExtractors.java  | 134 ++++++
 .../monitor/TestSuffixingNGramTokenizer.java       |  67 +++
 .../apache/lucene/monitor/TestTermPresearcher.java | 152 ++++++
 .../lucene/monitor/TestTermsEnumTokenFilter.java   |  73 +++
 .../monitor/TestWildcardTermPresearcher.java       |  80 ++++
 80 files changed, 8859 insertions(+)

diff --git a/dev-tools/idea/.idea/ant.xml b/dev-tools/idea/.idea/ant.xml
index d3f9655..c270538 100644
--- a/dev-tools/idea/.idea/ant.xml
+++ b/dev-tools/idea/.idea/ant.xml
@@ -25,6 +25,7 @@
     <buildFile url="file://$PROJECT_DIR$/lucene/highlighter/build.xml" />
     <buildFile url="file://$PROJECT_DIR$/lucene/join/build.xml" />
     <buildFile url="file://$PROJECT_DIR$/lucene/luke/build.xml" />
+    <buildFile url="file://$PROJECT_DIR$/lucene/luwak/build.xml" />
     <buildFile url="file://$PROJECT_DIR$/lucene/memory/build.xml" />
     <buildFile url="file://$PROJECT_DIR$/lucene/misc/build.xml" />
     <buildFile url="file://$PROJECT_DIR$/lucene/queries/build.xml" />
diff --git a/dev-tools/idea/.idea/modules.xml b/dev-tools/idea/.idea/modules.xml
index 9f8221c..faa7684 100644
--- a/dev-tools/idea/.idea/modules.xml
+++ b/dev-tools/idea/.idea/modules.xml
@@ -31,6 +31,7 @@
       <module group="Lucene/Other" filepath="$PROJECT_DIR$/lucene/highlighter/highlighter.iml" />
       <module group="Lucene/Other" filepath="$PROJECT_DIR$/lucene/join/join.iml" />
       <module group="Lucene/Other" filepath="$PROJECT_DIR$/lucene/luke/luke.iml" />
+      <module group="Lucene/Other" filepath="$PROJECT_DIR$/lucene/monitor/monitor.iml" />
       <module group="Lucene/Other" filepath="$PROJECT_DIR$/lucene/memory/memory.iml" />
       <module group="Lucene/Other" filepath="$PROJECT_DIR$/lucene/misc/misc.iml" />
       <module group="Lucene/Other" filepath="$PROJECT_DIR$/lucene/queries/queries.iml" />
diff --git a/dev-tools/idea/.idea/workspace.xml b/dev-tools/idea/.idea/workspace.xml
index bbc271e..493caa7 100644
--- a/dev-tools/idea/.idea/workspace.xml
+++ b/dev-tools/idea/.idea/workspace.xml
@@ -172,6 +172,14 @@
       <option name="TEST_SEARCH_SCOPE"><value defaultName="singleModule" /></option>
       <patterns><pattern testClass=".*\.Test[^.]*|.*\.[^.]*Test" /></patterns>
     </configuration>
+    <configuration default="false" name="Module monitor" type="JUnit" factoryName="JUnit">
+      <module name="monitor" />
+      <option name="TEST_OBJECT" value="pattern" />
+      <option name="WORKING_DIRECTORY" value="file://$PROJECT_DIR$/idea-build/lucene/monitor" />
+      <option name="VM_PARAMETERS" value="-ea -DtempDir=temp" />
+      <option name="TEST_SEARCH_SCOPE"><value defaultName="singleModule" /></option>
+      <patterns><pattern testClass=".*\.Test[^.]*|.*\.[^.]*Test" /></patterns>
+    </configuration>
     <configuration default="false" name="Module queries" type="JUnit" factoryName="JUnit">
       <module name="queries" />
       <option name="TEST_OBJECT" value="pattern" />
diff --git a/dev-tools/idea/lucene/monitor/monitor.iml b/dev-tools/idea/lucene/monitor/monitor.iml
new file mode 100644
index 0000000..5c63df4
--- /dev/null
+++ b/dev-tools/idea/lucene/monitor/monitor.iml
@@ -0,0 +1,32 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<module type="JAVA_MODULE" version="4">
+  <component name="NewModuleRootManager" inherit-compiler-output="false">
+    <output url="file://$MODULE_DIR$/../../idea-build/lucene/monitor/classes/java" />
+    <output-test url="file://$MODULE_DIR$/../../idea-build/lucene/monitor/classes/test" />
+    <exclude-output />
+    <content url="file://$MODULE_DIR$">
+      <sourceFolder url="file://$MODULE_DIR$/src/java" isTestSource="false" />
+      <sourceFolder url="file://$MODULE_DIR$/src/resources" isTestSource="false" />
+      <sourceFolder url="file://$MODULE_DIR$/src/test" isTestSource="true" />
+      <excludeFolder url="file://$MODULE_DIR$/work" />
+    </content>
+    <orderEntry type="inheritedJdk" />
+    <orderEntry type="sourceFolder" forTests="false" />
+    <orderEntry type="module-library">
+      <library>
+        <CLASSES>
+          <root url="file://$MODULE_DIR$/lib" />
+        </CLASSES>
+        <JAVADOC />
+        <SOURCES />
+        <jarDirectory url="file://$MODULE_DIR$/lib" recursive="false" />
+      </library>
+    </orderEntry>
+    <orderEntry type="library" scope="TEST" name="JUnit" level="project" />
+    <orderEntry type="module" scope="TEST" module-name="lucene-test-framework" />
+    <orderEntry type="module" module-name="lucene-core" />
+    <orderEntry type="module" module-name="analysis-common" />
+    <orderEntry type="module" module-name="memory" />
+    <orderEntry type="module" module-name="queryparser" />
+  </component>
+</module>
diff --git a/dev-tools/maven/lucene/monitor/pom.xml.template b/dev-tools/maven/lucene/monitor/pom.xml.template
new file mode 100644
index 0000000..3d915e9
--- /dev/null
+++ b/dev-tools/maven/lucene/monitor/pom.xml.template
@@ -0,0 +1,70 @@
+<!--
+  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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.lucene</groupId>
+    <artifactId>lucene-parent</artifactId>
+    <version>@version@</version>
+    <relativePath>../pom.xml</relativePath>
+  </parent>
+  <groupId>org.apache.lucene</groupId>
+  <artifactId>lucene-monitor</artifactId>
+  <packaging>jar</packaging>
+  <name>Lucene Monitor</name>
+  <description>
+    High-performance single-document index to compare against Query
+  </description>
+  <properties>
+    <module-directory>lucene/monitor</module-directory>
+    <relative-top-level>../../..</relative-top-level>
+    <module-path>${relative-top-level}/${module-directory}</module-path>
+  </properties>
+  <scm>
+    <connection>scm:git:${vc-anonymous-base-url}</connection>
+    <developerConnection>scm:git:${vc-dev-base-url}</developerConnection>
+    <url>${vc-browse-base-url};f=${module-directory}</url>
+  </scm>
+  <dependencies>
+    <dependency> 
+      <!-- lucene-test-framework dependency must be declared before lucene-core -->
+      <groupId>org.apache.lucene</groupId>
+      <artifactId>lucene-test-framework</artifactId>
+      <scope>test</scope>
+    </dependency>
+@lucene-monitor.internal.dependencies@
+@lucene-monitor.external.dependencies@
+@lucene-monitor.internal.test.dependencies@
+@lucene-monitor.external.test.dependencies@
+  </dependencies>
+  <build>
+    <sourceDirectory>${module-path}/src/java</sourceDirectory>
+    <testSourceDirectory>${module-path}/src/test</testSourceDirectory>
+    <testResources>
+      <testResource>
+        <directory>${project.build.testSourceDirectory}</directory>
+        <excludes>
+          <exclude>**/*.java</exclude>
+        </excludes>
+      </testResource>
+    </testResources>
+  </build>
+</project>
diff --git a/dev-tools/maven/lucene/pom.xml.template b/dev-tools/maven/lucene/pom.xml.template
index d0f98b0..1b721b5 100644
--- a/dev-tools/maven/lucene/pom.xml.template
+++ b/dev-tools/maven/lucene/pom.xml.template
@@ -55,6 +55,7 @@
     <module>join</module>
     <module>memory</module>
     <module>misc</module>
+    <module>monitor</module>
     <module>queries</module>
     <module>queryparser</module>
     <module>replicator</module>
diff --git a/lucene/monitor/build.xml b/lucene/monitor/build.xml
new file mode 100644
index 0000000..c378c1d
--- /dev/null
+++ b/lucene/monitor/build.xml
@@ -0,0 +1,53 @@
+<?xml version="1.0"?>
+
+<!--
+    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.
+ -->
+
+<project name="monitor" default="default">
+
+  <description>
+    Reverse-search implementation for monitoring and classification
+  </description>
+
+  <import file="../module-build.xml"/>
+
+  <path id="test.classpath">
+    <path refid="test.base.classpath"/>
+    <pathelement path="${memory.jar}"/>
+  </path>
+
+  <path id="classpath">
+    <pathelement path="${memory.jar}"/>
+    <pathelement path="${analyzers-common.jar}"/>
+    <pathelement path="${queryparser.jar}"/>
+    <path refid="base.classpath"/>
+  </path>
+
+  <target name="init" depends="module-build.init,jar-analyzers-common,jar-queryparser,jar-memory"/>
+
+  <target name="javadocs" depends="javadocs-memory,compile-core,check-javadocs-uptodate"
+          unless="javadocs-uptodate-${name}">
+    <invoke-module-javadoc>
+      <links>
+        <link href="../memory"/>
+        <link href="../analyzers-common"/>
+        <link href="../queryparser"/>
+      </links>
+    </invoke-module-javadoc>
+  </target>
+
+</project>
diff --git a/lucene/monitor/ivy.xml b/lucene/monitor/ivy.xml
new file mode 100644
index 0000000..9485d48
--- /dev/null
+++ b/lucene/monitor/ivy.xml
@@ -0,0 +1,21 @@
+<!--
+   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.    
+-->
+<ivy-module version="2.0">
+  <info organisation="org.apache.lucene" module="luwak"/>
+</ivy-module>
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/CandidateMatcher.java b/lucene/monitor/src/java/org/apache/lucene/monitor/CandidateMatcher.java
new file mode 100644
index 0000000..39b8ad5
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/CandidateMatcher.java
@@ -0,0 +1,133 @@
+/*
+ * 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.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+
+/**
+ * Class used to match candidate queries selected by a Presearcher from a Monitor
+ * query index.
+ */
+public abstract class CandidateMatcher<T extends QueryMatch> {
+
+  /**
+   * The searcher to run candidate queries against
+   */
+  protected final IndexSearcher searcher;
+
+  private final Map<String, Exception> errors = new HashMap<>();
+  private final List<MatchHolder<T>> matches;
+
+  private long searchTime = System.nanoTime();
+
+  private static class MatchHolder<T> {
+    Map<String, T> matches = new HashMap<>();
+  }
+
+  /**
+   * Creates a new CandidateMatcher for the supplied DocumentBatch
+   *
+   * @param searcher the IndexSearcher to run queries against
+   */
+  public CandidateMatcher(IndexSearcher searcher) {
+    this.searcher = searcher;
+    int docCount = searcher.getIndexReader().maxDoc();
+    this.matches = new ArrayList<>(docCount);
+    for (int i = 0; i < docCount; i++) {
+      this.matches.add(new MatchHolder<>());
+    }
+  }
+
+  /**
+   * Runs the supplied query against this CandidateMatcher's set of documents, storing any
+   * resulting match, and recording the query in the presearcher hits
+   *
+   * @param queryId    the query id
+   * @param matchQuery the query to run
+   * @param metadata   the query metadata
+   * @throws IOException on IO errors
+   */
+  protected abstract void matchQuery(String queryId, Query matchQuery, Map<String, String> metadata) throws IOException;
+
+  /**
+   * Record a match
+   *
+   * @param match a QueryMatch object
+   */
+  protected final void addMatch(T match, int doc) {
+    MatchHolder<T> docMatches = matches.get(doc);
+    docMatches.matches.compute(match.getQueryId(), (key, oldValue) -> {
+      if (oldValue != null) {
+        return resolve(match, oldValue);
+      }
+      return match;
+    });
+  }
+
+  /**
+   * If two matches from the same query are found (for example, two branches of a disjunction),
+   * combine them.
+   *
+   * @param match1 the first match found
+   * @param match2 the second match found
+   * @return a Match object that combines the two
+   */
+  public abstract T resolve(T match1, T match2);
+
+  /**
+   * Called by the Monitor if running a query throws an Exception
+   */
+  void reportError(String queryId, Exception e) {
+    this.errors.put(queryId, e);
+  }
+
+  /**
+   * @return the matches from this matcher
+   */
+  final MultiMatchingQueries<T> finish(long buildTime, int queryCount) {
+    doFinish();
+    this.searchTime = TimeUnit.MILLISECONDS.convert(System.nanoTime() - searchTime, TimeUnit.NANOSECONDS);
+    List<Map<String, T>> results = new ArrayList<>();
+    for (MatchHolder<T> matchHolder : matches) {
+      results.add(matchHolder.matches);
+    }
+    return new MultiMatchingQueries<>(results, errors, buildTime, searchTime, queryCount, matches.size());
+  }
+
+  /**
+   * Called when all monitoring of a batch of documents is complete
+   */
+  protected void doFinish() { }
+
+  /**
+   * Copy all matches from another CandidateMatcher
+   */
+  protected void copyMatches(CandidateMatcher<T> other) {
+    this.matches.clear();
+    this.matches.addAll(other.matches);
+  }
+
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/CollectingMatcher.java b/lucene/monitor/src/java/org/apache/lucene/monitor/CollectingMatcher.java
new file mode 100644
index 0000000..40a1957
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/CollectingMatcher.java
@@ -0,0 +1,84 @@
+/*
+ * 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.Map;
+
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorable;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.SimpleCollector;
+
+abstract class CollectingMatcher<T extends QueryMatch> extends CandidateMatcher<T> {
+
+  private final ScoreMode scoreMode;
+
+  CollectingMatcher(IndexSearcher searcher, ScoreMode scoreMode) {
+    super(searcher);
+    this.scoreMode = scoreMode;
+  }
+
+  @Override
+  protected void matchQuery(final String queryId, Query matchQuery, Map<String, String> metadata) throws IOException {
+    searcher.search(matchQuery, new MatchCollector(queryId, scoreMode));
+  }
+
+  /**
+   * Called when a query matches a Document
+   *
+   * @param queryId the query ID
+   * @param doc     the index of the document in the DocumentBatch
+   * @param scorer  the Scorer for this query
+   * @return a match object
+   * @throws IOException on IO error
+   */
+  protected abstract T doMatch(String queryId, int doc, Scorable scorer) throws IOException;
+
+  private class MatchCollector extends SimpleCollector {
+
+    private final String queryId;
+    private final ScoreMode scoreMode;
+    private Scorable scorer;
+
+    MatchCollector(String queryId, ScoreMode scoreMode) {
+      this.queryId = queryId;
+      this.scoreMode = scoreMode;
+    }
+
+    @Override
+    public void collect(int doc) throws IOException {
+      T match = doMatch(queryId, doc, scorer);
+      if (match != null) {
+        addMatch(match, doc);
+      }
+    }
+
+    @Override
+    public void setScorer(Scorable scorer) {
+      this.scorer = scorer;
+    }
+
+    @Override
+    public ScoreMode scoreMode() {
+      return scoreMode;
+    }
+  }
+
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/ConcurrentQueryLoader.java b/lucene/monitor/src/java/org/apache/lucene/monitor/ConcurrentQueryLoader.java
new file mode 100644
index 0000000..95520a7
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/ConcurrentQueryLoader.java
@@ -0,0 +1,196 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Objects;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.lucene.util.NamedThreadFactory;
+
+/**
+ * Utility class for concurrently loading queries into a Monitor.
+ * <p>
+ * This is useful to speed up startup times for a Monitor.  You can use multiple
+ * threads to parse and index queries before starting matches.
+ * <p>
+ * Use as follows:
+ * <pre class="prettyprint">
+ *     List&lt;QueryError&gt; errors = new ArrayList&lt;&gt;();
+ *     try (ConcurrentQueryLoader loader = new ConcurrentQueryLoader(monitor, errors)) {
+ *         for (MonitorQuery mq : getQueries()) {
+ *             loader.add(mq);
+ *         }
+ *     }
+ * </pre>
+ * <p>
+ * The Monitor's MonitorQueryParser must be thread-safe for this to work correctly.
+ */
+public class ConcurrentQueryLoader implements Closeable {
+
+  private final Monitor monitor;
+  private final ExecutorService executor;
+  private final CountDownLatch shutdownLatch;
+  private final BlockingQueue<MonitorQuery> queue;
+
+  private boolean shutdown = false;
+  private List<IOException> errors = new ArrayList<>();
+
+  public static final int DEFAULT_QUEUE_SIZE = 2000;
+
+  /**
+   * Create a new ConcurrentQueryLoader for a {@link Monitor}
+   *
+   * @param monitor Monitor
+   */
+  public ConcurrentQueryLoader(Monitor monitor) {
+    this(monitor, Runtime.getRuntime().availableProcessors(), DEFAULT_QUEUE_SIZE);
+  }
+
+  /**
+   * Create a new ConcurrentQueryLoader
+   *
+   * @param monitor   the Monitor to load queries to
+   * @param threads   the number of threads to use
+   * @param queueSize the size of the buffer to hold queries in
+   */
+  public ConcurrentQueryLoader(Monitor monitor, int threads, int queueSize) {
+    this.monitor = monitor;
+    this.queue = new LinkedBlockingQueue<>(queueSize);
+    this.executor = Executors.newFixedThreadPool(threads, new NamedThreadFactory("loader"));
+    this.shutdownLatch = new CountDownLatch(threads);
+    for (int i = 0; i < threads; i++) {
+      this.executor.submit(new Worker(queueSize / threads));
+    }
+  }
+
+  /**
+   * Add a MonitorQuery to the loader's internal buffer
+   * <p>
+   * If the buffer is full, this will block until there is room to add
+   * the MonitorQuery
+   *
+   * @param mq the monitor query
+   * @throws InterruptedException if interrupted while waiting
+   */
+  public void add(MonitorQuery mq) throws InterruptedException {
+    if (shutdown)
+      throw new IllegalStateException("ConcurrentQueryLoader has been shutdown, cannot add new queries");
+    this.queue.put(mq);
+  }
+
+  @Override
+  public void close() throws IOException {
+    this.shutdown = true;
+    this.executor.shutdown();
+    try {
+      this.shutdownLatch.await();
+    } catch (InterruptedException e) {
+      // fine
+    }
+    if (errors.size() > 0) {
+      IOException e = new IOException();
+      errors.forEach(e::addSuppressed);
+      throw e;
+    }
+  }
+
+  private class Worker implements Runnable {
+
+    final List<MonitorQuery> workerQueue;
+    final int queueSize;
+    boolean running = true;
+
+    Worker(int queueSize) {
+      workerQueue = new ArrayList<>(queueSize);
+      this.queueSize = queueSize;
+    }
+
+    @Override
+    public void run() {
+      try {
+        while (running) {
+          workerQueue.clear();
+          drain(queue, workerQueue, queueSize, 100, TimeUnit.MILLISECONDS);
+          if (workerQueue.size() == 0 && shutdown)
+            running = false;
+          if (workerQueue.size() > 0) {
+            monitor.register(workerQueue);
+          }
+        }
+      } catch (IOException e) {
+        errors.add(e);
+      } catch (InterruptedException e) {
+        Thread.currentThread().interrupt();
+      } finally {
+        shutdownLatch.countDown();
+      }
+    }
+  }
+
+  /**
+   * Drains the queue as {@link BlockingQueue#drainTo(Collection, int)}, but if the requested
+   * {@code numElements} elements are not available, it will wait for them up to the specified
+   * timeout.
+   * <p>
+   * Taken from Google Guava 18.0 Queues
+   *
+   * @param q           the blocking queue to be drained
+   * @param buffer      where to add the transferred elements
+   * @param numElements the number of elements to be waited for
+   * @param timeout     how long to wait before giving up, in units of {@code unit}
+   * @param unit        a {@code TimeUnit} determining how to interpret the timeout parameter
+   * @param <E>         the type of the queue
+   * @return the number of elements transferred
+   * @throws InterruptedException if interrupted while waiting
+   */
+  private static <E> int drain(BlockingQueue<E> q, Collection<? super E> buffer, int numElements,
+                              long timeout, TimeUnit unit) throws InterruptedException {
+    buffer = Objects.requireNonNull(buffer);
+    /*
+     * This code performs one System.nanoTime() more than necessary, and in return, the time to
+     * execute Queue#drainTo is not added *on top* of waiting for the timeout (which could make
+     * the timeout arbitrarily inaccurate, given a queue that is slow to drain).
+     */
+    long deadline = System.nanoTime() + unit.toNanos(timeout);
+    int added = 0;
+    while (added < numElements) {
+      // we could rely solely on #poll, but #drainTo might be more efficient when there are multiple
+      // elements already available (e.g. LinkedBlockingQueue#drainTo locks only once)
+      added += q.drainTo(buffer, numElements - added);
+      if (added < numElements) { // not enough elements immediately available; will have to poll
+        E e = q.poll(deadline - System.nanoTime(), TimeUnit.NANOSECONDS);
+        if (e == null) {
+          break; // we already waited enough, and there are no more elements in sight
+        }
+        buffer.add(e);
+        added++;
+      }
+    }
+    return added;
+  }
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/CustomQueryHandler.java b/lucene/monitor/src/java/org/apache/lucene/monitor/CustomQueryHandler.java
new file mode 100644
index 0000000..2e24043
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/CustomQueryHandler.java
@@ -0,0 +1,45 @@
+/*
+ * 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 org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.search.Query;
+
+/**
+ * Builds a {@link QueryTree} for a query that needs custom treatment
+ *
+ * The default query analyzers will use the QueryVisitor API to extract
+ * terms from queries.  If different handling is needed, implement a
+ * CustomQueryHandler and pass it to the presearcher
+ */
+public interface CustomQueryHandler {
+
+  /**
+   * Builds a {@link QueryTree} node from a query
+   */
+  QueryTree handleQuery(Query query, TermWeightor termWeightor);
+
+  /**
+   * Adds additional processing to the {@link TokenStream} over a document's
+   * terms index
+   */
+  default TokenStream wrapTermStream(String field, TokenStream in) {
+    return in;
+  }
+
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/DocumentBatch.java b/lucene/monitor/src/java/org/apache/lucene/monitor/DocumentBatch.java
new file mode 100644
index 0000000..69119ef
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/DocumentBatch.java
@@ -0,0 +1,121 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.function.Supplier;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.memory.MemoryIndex;
+import org.apache.lucene.store.ByteBuffersDirectory;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+
+abstract class DocumentBatch implements Closeable, Supplier<LeafReader> {
+
+  /**
+   * Create a DocumentBatch containing a single InputDocument
+   *
+   * @param doc the document to add
+   * @return the batch containing the input document
+   */
+  public static DocumentBatch of(Analyzer analyzer, Document doc) {
+    return new SingletonDocumentBatch(analyzer, doc);
+  }
+
+  /**
+   * Create a DocumentBatch containing a set of InputDocuments
+   *
+   * @param docs Collection of documents to add
+   * @return the batch containing the input documents
+   */
+  public static DocumentBatch of(Analyzer analyzer, Document... docs) {
+    return new MultiDocumentBatch(analyzer, docs);
+  }
+
+  // Implementation of DocumentBatch for collections of documents
+  private static class MultiDocumentBatch extends DocumentBatch {
+
+    private final Directory directory = new ByteBuffersDirectory();
+    private final LeafReader reader;
+
+    MultiDocumentBatch(Analyzer analyzer, Document... docs) {
+      IndexWriterConfig iwc = new IndexWriterConfig(analyzer);
+      try (IndexWriter writer = new IndexWriter(directory, iwc)) {
+        this.reader = build(writer, docs);
+      } catch (IOException e) {
+        throw new RuntimeException(e);  // This is a RAMDirectory, so should never happen...
+      }
+    }
+
+    @Override
+    public LeafReader get() {
+      return reader;
+    }
+
+    private LeafReader build(IndexWriter writer, Document... docs) throws IOException {
+      writer.addDocuments(Arrays.asList(docs));
+      writer.commit();
+      writer.forceMerge(1);
+      LeafReader reader = DirectoryReader.open(directory).leaves().get(0).reader();
+      assert reader != null;
+      return reader;
+    }
+
+    @Override
+    public void close() throws IOException {
+      IOUtils.close(reader, directory);
+    }
+
+  }
+
+  // Specialized class for batches containing a single object - MemoryIndex benchmarks as
+  // better performing than RAMDirectory for this case
+  private static class SingletonDocumentBatch extends DocumentBatch {
+
+    private final LeafReader reader;
+
+    private SingletonDocumentBatch(Analyzer analyzer, Document doc) {
+      MemoryIndex memoryindex = new MemoryIndex(true, true);
+      for (IndexableField field : doc) {
+        memoryindex.addField(field, analyzer);
+      }
+      memoryindex.freeze();
+      reader = (LeafReader) memoryindex.createSearcher().getIndexReader();
+    }
+
+    @Override
+    public LeafReader get() {
+      return reader;
+    }
+
+    @Override
+    public void close() throws IOException {
+      reader.close();
+    }
+  }
+
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/ExplainingMatch.java b/lucene/monitor/src/java/org/apache/lucene/monitor/ExplainingMatch.java
new file mode 100644
index 0000000..adcc667
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/ExplainingMatch.java
@@ -0,0 +1,81 @@
+/*
+ * 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.Map;
+import java.util.Objects;
+
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.Query;
+
+/**
+ * A query match containing the score explanation of the match
+ */
+public class ExplainingMatch extends QueryMatch {
+
+  /**
+   * A MatcherFactory for producing ExplainingMatches
+   */
+  public static final MatcherFactory<ExplainingMatch> MATCHER = searcher -> new CandidateMatcher<ExplainingMatch>(searcher) {
+    @Override
+    protected void matchQuery(String queryId, Query matchQuery, Map<String, String> metadata) throws IOException {
+      int maxDocs = searcher.getIndexReader().maxDoc();
+      for (int i = 0; i < maxDocs; i++) {
+        Explanation explanation = searcher.explain(matchQuery, i);
+        if (explanation.isMatch())
+          addMatch(new ExplainingMatch(queryId, explanation), i);
+      }
+    }
+
+    @Override
+    public ExplainingMatch resolve(ExplainingMatch match1, ExplainingMatch match2) {
+      return new ExplainingMatch(match1.getQueryId(),
+          Explanation.match(match1.getExplanation().getValue().doubleValue() + match2.getExplanation().getValue().doubleValue(),
+              "sum of:", match1.getExplanation(), match2.getExplanation()));
+    }
+  };
+
+  private final Explanation explanation;
+
+  ExplainingMatch(String queryId, Explanation explanation) {
+    super(queryId);
+    this.explanation = explanation;
+  }
+
+  /**
+   * @return the Explanation
+   */
+  public Explanation getExplanation() {
+    return explanation;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    if (!super.equals(o)) return false;
+    ExplainingMatch that = (ExplainingMatch) o;
+    return Objects.equals(explanation, that.explanation);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(super.hashCode(), explanation);
+  }
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/ForceNoBulkScoringQuery.java b/lucene/monitor/src/java/org/apache/lucene/monitor/ForceNoBulkScoringQuery.java
new file mode 100644
index 0000000..7b1078a
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/ForceNoBulkScoringQuery.java
@@ -0,0 +1,103 @@
+/*
+ * 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.Objects;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.*;
+import org.apache.lucene.search.Matches;
+
+/**
+ * Query wrapper that forces its wrapped Query to use the default doc-by-doc
+ * BulkScorer.
+ */
+class ForceNoBulkScoringQuery extends Query {
+
+  private final Query inner;
+
+  public ForceNoBulkScoringQuery(Query inner) {
+    this.inner = inner;
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    Query rewritten = inner.rewrite(reader);
+    if (rewritten != inner)
+      return new ForceNoBulkScoringQuery(rewritten);
+    return super.rewrite(reader);
+  }
+
+  @Override
+  public void visit(QueryVisitor visitor) {
+    inner.visit(visitor);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    ForceNoBulkScoringQuery that = (ForceNoBulkScoringQuery) o;
+    return Objects.equals(inner, that.inner);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(inner);
+  }
+
+  public Query getWrappedQuery() {
+    return inner;
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, ScoreMode scoreMode, float boost) throws IOException {
+
+    final Weight innerWeight = inner.createWeight(searcher, scoreMode, boost);
+
+    return new Weight(ForceNoBulkScoringQuery.this) {
+
+      @Override
+      public boolean isCacheable(LeafReaderContext ctx) {
+        return innerWeight.isCacheable(ctx);
+      }
+
+      @Override
+      public Explanation explain(LeafReaderContext leafReaderContext, int i) throws IOException {
+        return innerWeight.explain(leafReaderContext, i);
+      }
+
+      @Override
+      public Scorer scorer(LeafReaderContext leafReaderContext) throws IOException {
+        return innerWeight.scorer(leafReaderContext);
+      }
+
+      @Override
+      public Matches matches(LeafReaderContext context, int doc) throws IOException {
+        return innerWeight.matches(context, doc);
+      }
+    };
+  }
+
+  @Override
+  public String toString(String s) {
+    return "NoBulkScorer(" + inner.toString(s) + ")";
+  }
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/HighlightsMatch.java b/lucene/monitor/src/java/org/apache/lucene/monitor/HighlightsMatch.java
new file mode 100644
index 0000000..74b0332
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/HighlightsMatch.java
@@ -0,0 +1,230 @@
+/*
+ * 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 org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.Matches;
+import org.apache.lucene.search.MatchesIterator;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Weight;
+
+/**
+ * QueryMatch object that contains the hit positions of a matching Query
+ * <p>
+ * If the Query does not support interval iteration (eg, if it gets re-written to
+ * a Filter), then no hits will be reported, but an IntervalsQueryMatch will still
+ * be returned from an IntervalsMatcher to indicate a match.
+ */
+public class HighlightsMatch extends QueryMatch {
+
+  public static final MatcherFactory<HighlightsMatch> MATCHER = searcher -> new CandidateMatcher<HighlightsMatch>(searcher) {
+
+    @Override
+    protected void matchQuery(String queryId, Query matchQuery, Map<String, String> metadata) throws IOException {
+      Weight w = searcher.createWeight(searcher.rewrite(matchQuery), ScoreMode.COMPLETE_NO_SCORES, 1);
+      for (LeafReaderContext ctx : searcher.getIndexReader().leaves()) {
+        for (int i = 0; i < ctx.reader().maxDoc(); i++) {
+          Matches matches = w.matches(ctx, i);
+          if (matches != null) {
+            addMatch(buildMatch(matches, queryId), i);
+          }
+        }
+      }
+    }
+
+    @Override
+    public HighlightsMatch resolve(HighlightsMatch match1, HighlightsMatch match2) {
+      return HighlightsMatch.merge(match1.getQueryId(), match1, match2);
+    }
+
+    private HighlightsMatch buildMatch(Matches matches, String queryId) throws IOException {
+      HighlightsMatch m = new HighlightsMatch(queryId);
+      for (String field : matches) {
+        MatchesIterator mi = matches.getMatches(field);
+        while (mi.next()) {
+          MatchesIterator sub = mi.getSubMatches();
+          if (sub != null) {
+            while (sub.next()) {
+              m.addHit(field, sub.startPosition(), sub.endPosition(), sub.startOffset(), sub.endOffset());
+            }
+          }
+          else {
+            m.addHit(field, mi.startPosition(), mi.endPosition(), mi.startOffset(), mi.endOffset());
+          }
+        }
+      }
+      return m;
+    }
+  };
+
+  private final Map<String, Set<Hit>> hits;
+
+  HighlightsMatch(String queryId) {
+    super(queryId);
+    this.hits = new TreeMap<>();
+  }
+
+  /**
+   * @return a map of hits per field
+   */
+  public Map<String, Set<Hit>> getHits() {
+    return Collections.unmodifiableMap(this.hits);
+  }
+
+  /**
+   * @return the fields in which matches have been found
+   */
+  public Set<String> getFields() {
+    return Collections.unmodifiableSet(hits.keySet());
+  }
+
+  /**
+   * Get the hits for a specific field
+   *
+   * @param field the field
+   * @return the Hits found in this field
+   */
+  public Collection<Hit> getHits(String field) {
+    Collection<Hit> found = hits.get(field);
+    if (found != null)
+      return Collections.unmodifiableCollection(found);
+    return Collections.emptyList();
+  }
+
+  /**
+   * @return the total number of hits for the query
+   */
+  public int getHitCount() {
+    int c = 0;
+    for (Set<Hit> fieldhits : hits.values()) {
+      c += fieldhits.size();
+    }
+    return c;
+  }
+
+  static HighlightsMatch merge(String queryId, HighlightsMatch... matches) {
+    HighlightsMatch newMatch = new HighlightsMatch(queryId);
+    for (HighlightsMatch match : matches) {
+      for (String field : match.getFields()) {
+        Set<Hit> hitSet = newMatch.hits.computeIfAbsent(field, f -> new TreeSet<>());
+        hitSet.addAll(match.getHits(field));
+      }
+    }
+    return newMatch;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (!(o instanceof HighlightsMatch)) return false;
+    if (!super.equals(o)) return false;
+
+    HighlightsMatch that = (HighlightsMatch) o;
+
+    if (hits != null ? !hits.equals(that.hits) : that.hits != null) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    result = 31 * result + (hits != null ? hits.hashCode() : 0);
+    return result;
+  }
+
+  @Override
+  public String toString() {
+    return super.toString() + "{hits=" + hits + "}";
+  }
+
+  void addHit(String field, int startPos, int endPos, int startOffset, int endOffset) {
+    Set<Hit> hitSet = hits.computeIfAbsent(field, f -> new TreeSet<>());
+    hitSet.add(new Hit(startPos, startOffset, endPos, endOffset));
+  }
+
+  /**
+   * Represents an individual hit
+   */
+  public static class Hit implements Comparable<Hit> {
+
+    /**
+     * The start position
+     */
+    public final int startPosition;
+
+    /**
+     * The start offset
+     */
+    public final int startOffset;
+
+    /**
+     * The end positions
+     */
+    public final int endPosition;
+
+    /**
+     * The end offset
+     */
+    public final int endOffset;
+
+    public Hit(int startPosition, int startOffset, int endPosition, int endOffset) {
+      this.startPosition = startPosition;
+      this.startOffset = startOffset;
+      this.endPosition = endPosition;
+      this.endOffset = endOffset;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) return true;
+      if (!(obj instanceof Hit))
+        return false;
+      Hit other = (Hit) obj;
+      return this.startOffset == other.startOffset &&
+          this.endOffset == other.endOffset &&
+          this.startPosition == other.startPosition &&
+          this.endPosition == other.endPosition;
+    }
+
+    @Override
+    public int hashCode() {
+      int result = startPosition;
+      result = 31 * result + startOffset;
+      result = 31 * result + endPosition;
+      result = 31 * result + endOffset;
+      return result;
+    }
+
+    @Override
+    public String toString() {
+      return String.format(Locale.ROOT, "%d(%d)->%d(%d)", startPosition, startOffset, endPosition, endOffset);
+    }
+
+    @Override
+    public int compareTo(Hit other) {
+      if (this.startPosition != other.startPosition)
+        return Integer.compare(this.startPosition, other.startPosition);
+      return Integer.compare(this.endPosition, other.endPosition);
+    }
+  }
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/MatcherFactory.java b/lucene/monitor/src/java/org/apache/lucene/monitor/MatcherFactory.java
new file mode 100644
index 0000000..5870c09
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/MatcherFactory.java
@@ -0,0 +1,35 @@
+/*
+ * 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 org.apache.lucene.search.IndexSearcher;
+
+/**
+ * Interface for the creation of new CandidateMatcher objects
+ *
+ * @param <T> a subclass of {@link CandidateMatcher}
+ */
+public interface MatcherFactory<T extends QueryMatch> {
+
+  /**
+   * Create a new {@link CandidateMatcher} object, to select
+   * queries to match against the passed-in IndexSearcher
+   */
+  CandidateMatcher<T> createMatcher(IndexSearcher searcher);
+
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/MatchingQueries.java b/lucene/monitor/src/java/org/apache/lucene/monitor/MatchingQueries.java
new file mode 100644
index 0000000..2ae927d
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/MatchingQueries.java
@@ -0,0 +1,97 @@
+/*
+ * 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.util.*;
+
+/**
+ * Class to hold the results of matching a single {@link org.apache.lucene.document.Document}
+ * against queries held in the Monitor
+ *
+ * @param <T> the type of QueryMatch returned
+ */
+public class MatchingQueries<T extends QueryMatch> {
+
+  private final Map<String, T> matches;
+  private final Map<String, Exception> errors;
+
+  private final long queryBuildTime;
+  private final long searchTime;
+  private final int queriesRun;
+
+  MatchingQueries(Map<String, T> matches, Map<String, Exception> errors,
+                  long queryBuildTime, long searchTime, int queriesRun) {
+    this.matches = Collections.unmodifiableMap(matches);
+    this.errors = Collections.unmodifiableMap(errors);
+    this.queryBuildTime = queryBuildTime;
+    this.searchTime = searchTime;
+    this.queriesRun = queriesRun;
+  }
+
+  /**
+   * Returns the QueryMatch for the given query, or null if it did not match
+   *
+   * @param queryId the query id
+   */
+  public T matches(String queryId) {
+    return matches.get(queryId);
+  }
+
+  /**
+   * @return all matches
+   */
+  public Collection<T> getMatches() {
+    return matches.values();
+  }
+
+  /**
+   * @return the number of queries that matched
+   */
+  public int getMatchCount() {
+    return matches.size();
+  }
+
+  /**
+   * @return how long (in ms) it took to build the Presearcher query for the matcher run
+   */
+  public long getQueryBuildTime() {
+    return queryBuildTime;
+  }
+
+  /**
+   * @return how long (in ms) it took to run the selected queries
+   */
+  public long getSearchTime() {
+    return searchTime;
+  }
+
+  /**
+   * @return the number of queries passed to this CandidateMatcher during the matcher run
+   */
+  public int getQueriesRun() {
+    return queriesRun;
+  }
+
+  /**
+   * @return a List of any MatchErrors created during the matcher run
+   */
+  public Map<String, Exception> getErrors() {
+    return errors;
+  }
+
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/Monitor.java b/lucene/monitor/src/java/org/apache/lucene/monitor/Monitor.java
new file mode 100644
index 0000000..edde451
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/Monitor.java
@@ -0,0 +1,409 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Matches;
+import org.apache.lucene.search.MatchesIterator;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.NamedThreadFactory;
+
+/**
+ * A Monitor contains a set of {@link Query} objects with associated IDs, and efficiently
+ * matches them against sets of {@link Document} objects.
+ */
+public class Monitor implements Closeable {
+
+  protected final Presearcher presearcher;
+  private final Analyzer analyzer;
+
+  private final QueryIndex queryIndex;
+
+  private final List<MonitorUpdateListener> listeners = new ArrayList<>();
+
+  private final long commitBatchSize;
+
+  private final ScheduledExecutorService purgeExecutor;
+
+  private long lastPurged = -1;
+
+  /**
+   * Create a non-persistent Monitor instance with the default term-filtering Presearcher
+   *
+   * @param analyzer to analyze {@link Document}s at match time
+   */
+  public Monitor(Analyzer analyzer) throws IOException {
+    this(analyzer, new TermFilteredPresearcher());
+  }
+
+  /**
+   * Create a new non-persistent Monitor instance
+   *
+   * @param analyzer to analyze {@link Document}s at match time
+   * @param presearcher the presearcher to use
+   */
+  public Monitor(Analyzer analyzer, Presearcher presearcher) throws IOException {
+    this(analyzer, presearcher, new MonitorConfiguration());
+  }
+
+  /**
+   * Create a new Monitor instance with a specific configuration
+   *
+   * @param analyzer to analyze {@link Document}s at match time
+   * @param config   the configuration
+   */
+  public Monitor(Analyzer analyzer, MonitorConfiguration config) throws IOException {
+    this(analyzer, new TermFilteredPresearcher(), config);
+  }
+
+  /**
+   * Create a new Monitor instance
+   *
+   * @param analyzer      to analyze {@link Document}s at match time
+   * @param presearcher   the presearcher to use
+   * @param configuration the configuration
+   */
+  public Monitor(Analyzer analyzer, Presearcher presearcher,
+                 MonitorConfiguration configuration) throws IOException {
+
+    this.analyzer = analyzer;
+    this.presearcher = presearcher;
+    this.queryIndex = new QueryIndex(configuration, presearcher);
+
+    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());
+
+    this.commitBatchSize = configuration.getQueryUpdateBufferSize();
+  }
+
+  /**
+   * Register a {@link MonitorUpdateListener} that will be notified whenever changes
+   * are made to the Monitor's queryindex
+   *
+   * @param listener listener to register
+   */
+  public void addQueryIndexUpdateListener(MonitorUpdateListener listener) {
+    listeners.add(listener);
+  }
+
+  /**
+   * @return Statistics for the internal query index and cache
+   */
+  public QueryCacheStats getQueryCacheStats() {
+    return new QueryCacheStats(queryIndex.numDocs(), queryIndex.cacheSize(), lastPurged);
+  }
+
+  /**
+   * Statistics for the query cache and query index
+   */
+  public static class QueryCacheStats {
+
+    /**
+     * Total number of queries in the query index
+     */
+    public final int queries;
+
+    /**
+     * Total number of queries int the query cache
+     */
+    public final int cachedQueries;
+
+    /**
+     * Time the query cache was last purged
+     */
+    public final long lastPurged;
+
+    public QueryCacheStats(int queries, int cachedQueries, long lastPurged) {
+      this.queries = queries;
+      this.cachedQueries = cachedQueries;
+      this.lastPurged = lastPurged;
+    }
+  }
+
+  /**
+   * 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
+   */
+  public void purgeCache() throws IOException {
+    queryIndex.purgeCache();
+    lastPurged = System.nanoTime();
+    listeners.forEach(MonitorUpdateListener::onPurge);
+  }
+
+  @Override
+  public void close() throws IOException {
+    purgeExecutor.shutdown();
+    queryIndex.close();
+  }
+
+  /**
+   * Add new queries to the monitor
+   *
+   * @param queries the MonitorQueries to add
+   */
+  public void register(Iterable<MonitorQuery> queries) throws IOException {
+    List<MonitorQuery> updates = new ArrayList<>();
+    for (MonitorQuery query : queries) {
+      updates.add(query);
+      if (updates.size() > commitBatchSize) {
+        commit(updates);
+        updates.clear();
+      }
+    }
+    commit(updates);
+  }
+
+  private void commit(List<MonitorQuery> updates) throws IOException {
+    queryIndex.commit(updates);
+    listeners.forEach(l -> l.afterUpdate(updates));
+  }
+
+  /**
+   * Add new queries to the monitor
+   *
+   * @param queries the MonitorQueries to add
+   * @throws IOException     on IO errors
+   */
+  public void register(MonitorQuery... queries) throws IOException {
+    register(Arrays.asList(queries));
+  }
+
+  /**
+   * Delete queries from the monitor by ID
+   *
+   * @param queryIds the IDs to delete
+   * @throws IOException on IO errors
+   */
+  public void deleteById(List<String> queryIds) throws IOException {
+    queryIndex.deleteQueries(queryIds);
+    listeners.forEach(l -> l.afterDelete(queryIds));
+  }
+
+  /**
+   * Delete queries from the monitor by ID
+   *
+   * @param queryIds the IDs to delete
+   * @throws IOException on IO errors
+   */
+  public void deleteById(String... queryIds) throws IOException {
+    deleteById(Arrays.asList(queryIds));
+  }
+
+  /**
+   * Delete all queries from the monitor
+   *
+   * @throws IOException on IO errors
+   */
+  public void clear() throws IOException {
+    queryIndex.clear();
+    listeners.forEach(MonitorUpdateListener::afterClear);
+  }
+
+  /**
+   * Match an array of {@link Document}s against the queryindex, calling a {@link CandidateMatcher} produced by the
+   * supplied {@link MatcherFactory} for each possible matching query.
+   *
+   * @param docs    the DocumentBatch to match
+   * @param factory a {@link MatcherFactory} to use to create a {@link CandidateMatcher} for the match run
+   * @param <T>     the type of {@link QueryMatch} to return
+   * @return a {@link MatchingQueries} object summarizing the match run.
+   * @throws IOException on IO errors
+   */
+  public <T extends QueryMatch> MultiMatchingQueries<T> match(Document[] docs, MatcherFactory<T> factory) throws IOException {
+    try (DocumentBatch batch = DocumentBatch.of(analyzer, docs)) {
+      LeafReader reader = batch.get();
+      CandidateMatcher<T> matcher = factory.createMatcher(new IndexSearcher(batch.get()));
+      StandardQueryCollector<T> collector = new StandardQueryCollector<>(matcher);
+      long buildTime = queryIndex.search(t -> presearcher.buildQuery(reader, t), collector);
+      return matcher.finish(buildTime, collector.queryCount);
+    }
+  }
+
+  /**
+   * Match a single {@link Document} against the queryindex, calling a {@link CandidateMatcher} produced by the
+   * supplied {@link MatcherFactory} for each possible matching query.
+   *
+   * @param doc     the InputDocument to match
+   * @param factory a {@link MatcherFactory} to use to create a {@link CandidateMatcher} for the match run
+   * @param <T>     the type of {@link QueryMatch} to return
+   * @return a {@link MatchingQueries} object summarizing the match run.
+   * @throws IOException on IO errors
+   */
+  public <T extends QueryMatch> MatchingQueries<T> match(Document doc, MatcherFactory<T> factory) throws IOException {
+    return match(new Document[]{ doc }, factory).singleton();
+  }
+
+  /**
+   * Get the MonitorQuery for a given query id
+   *
+   * @param queryId the id of the query to get
+   * @return the MonitorQuery stored for this id, or null if not found
+   * @throws IOException           on IO errors
+   * @throws IllegalStateException if queries are not stored in the queryindex
+   */
+  public MonitorQuery getQuery(final String queryId) throws IOException {
+    return queryIndex.getQuery(queryId);
+  }
+
+  /**
+   * @return the number of queries (after decomposition) stored in this Monitor
+   */
+  public int getDisjunctCount() {
+    return queryIndex.numDocs();
+  }
+
+  /**
+   * @return the number of queries stored in this Monitor
+   * @throws IOException on IO errors
+   */
+  public int getQueryCount() throws IOException {
+    return getQueryIds().size();
+  }
+
+  /**
+   * @return the set of query ids of the queries stored in this Monitor
+   * @throws IOException on IO errors
+   */
+  public Set<String> getQueryIds() throws IOException {
+    final Set<String> ids = new HashSet<>();
+    queryIndex.scan((id, query, dataValues) -> ids.add(id));
+    return ids;
+  }
+
+  // For each query selected by the presearcher, pass on to a CandidateMatcher
+  private static class StandardQueryCollector<T extends QueryMatch> implements QueryIndex.QueryCollector {
+
+    final CandidateMatcher<T> matcher;
+    int queryCount = 0;
+
+    private StandardQueryCollector(CandidateMatcher<T> matcher) {
+      this.matcher = matcher;
+    }
+
+    @Override
+    public void matchQuery(String id, QueryCacheEntry query, QueryIndex.DataValues dataValues) throws IOException {
+      if (query == null)
+        return;
+      try {
+        queryCount++;
+        matcher.matchQuery(id, query.matchQuery, query.metadata);
+      } catch (Exception e) {
+        matcher.reportError(id, e);
+      }
+    }
+
+  }
+
+  /**
+   * Match a DocumentBatch against the queries stored in the Monitor, also returning information
+   * about which queries were selected by the presearcher, and why.
+   *
+   * @param docs    a DocumentBatch to match against the index
+   * @param factory a {@link MatcherFactory} to use to create a {@link CandidateMatcher} for the match run
+   * @param <T>     the type of QueryMatch produced by the CandidateMatcher
+   * @return a {@link PresearcherMatches} object containing debug information
+   * @throws IOException on IO errors
+   */
+  public <T extends QueryMatch> PresearcherMatches<T> debug(Document[] docs, MatcherFactory<T> factory)
+      throws IOException {
+    try (DocumentBatch batch = DocumentBatch.of(analyzer, docs)) {
+      LeafReader reader = batch.get();
+      IndexSearcher searcher = new IndexSearcher(reader);
+      searcher.setQueryCache(null);
+      PresearcherQueryCollector<T> collector = new PresearcherQueryCollector<>(factory.createMatcher(searcher));
+      long buildTime = queryIndex.search(t -> new ForceNoBulkScoringQuery(presearcher.buildQuery(reader, t)), collector);
+      return collector.getMatches(buildTime);
+    }
+  }
+
+  /**
+   * Match a single {@link Document} against the queries stored in the Monitor, also returning information
+   * about which queries were selected by the presearcher, and why.
+   *
+   * @param doc     an InputDocument to match against the index
+   * @param factory a {@link MatcherFactory} to use to create a {@link CandidateMatcher} for the match run
+   * @param <T>     the type of QueryMatch produced by the CandidateMatcher
+   * @return a {@link PresearcherMatches} object containing debug information
+   * @throws IOException on IO errors
+   */
+  public <T extends QueryMatch> PresearcherMatches<T> debug(Document doc, MatcherFactory<T> factory) throws IOException {
+    return debug(new Document[]{doc}, factory);
+  }
+
+  private class PresearcherQueryCollector<T extends QueryMatch> extends StandardQueryCollector<T> {
+
+    final Map<String, StringBuilder> matchingTerms = new HashMap<>();
+
+    private PresearcherQueryCollector(CandidateMatcher<T> matcher) {
+      super(matcher);
+    }
+
+    public PresearcherMatches<T> getMatches(long buildTime) {
+      return new PresearcherMatches<>(matchingTerms, matcher.finish(buildTime, queryCount));
+    }
+
+    @Override
+    public ScoreMode scoreMode() {
+      return ScoreMode.COMPLETE;
+    }
+
+    @Override
+    public void matchQuery(final String id, QueryCacheEntry query, QueryIndex.DataValues dataValues) throws IOException {
+      Weight w = ((Scorer)dataValues.scorer).getWeight();
+      Matches matches = w.matches(dataValues.ctx, dataValues.scorer.docID());
+      for (String field : matches) {
+        MatchesIterator mi = matches.getMatches(field);
+        while (mi.next()) {
+          matchingTerms.computeIfAbsent(id, i -> new StringBuilder())
+              .append(" ").append(mi.getQuery());
+        }
+      }
+      super.matchQuery(id, query, dataValues);
+    }
+
+  }
+
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/MonitorConfiguration.java b/lucene/monitor/src/java/org/apache/lucene/monitor/MonitorConfiguration.java
new file mode 100644
index 0000000..48f0c91
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/MonitorConfiguration.java
@@ -0,0 +1,135 @@
+/*
+ * 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.concurrent.TimeUnit;
+
+import org.apache.lucene.analysis.core.KeywordAnalyzer;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.TieredMergePolicy;
+import org.apache.lucene.store.ByteBuffersDirectory;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FSDirectory;
+
+/**
+ * Encapsulates various configuration settings for a Monitor's query index
+ */
+public class MonitorConfiguration {
+
+  private int queryUpdateBufferSize = 5000;
+  private long purgeFrequency = 5;
+  private TimeUnit purgeFrequencyUnits = TimeUnit.MINUTES;
+  private QueryDecomposer queryDecomposer = new QueryDecomposer();
+  private Path indexPath = null;
+  private MonitorQuerySerializer serializer;
+
+  private static IndexWriterConfig defaultIndexWriterConfig() {
+    IndexWriterConfig iwc = new IndexWriterConfig(new KeywordAnalyzer());
+    TieredMergePolicy mergePolicy = new TieredMergePolicy();
+    mergePolicy.setSegmentsPerTier(4);
+    iwc.setMergePolicy(mergePolicy);
+    iwc.setOpenMode(IndexWriterConfig.OpenMode.CREATE_OR_APPEND);
+    return iwc;
+  }
+
+  public MonitorConfiguration setIndexPath(Path indexPath, MonitorQuerySerializer serializer) {
+    this.indexPath = indexPath;
+    this.serializer = serializer;
+    return this;
+  }
+
+  public IndexWriter buildIndexWriter() throws IOException {
+    Directory directory = indexPath == null ? new ByteBuffersDirectory() : FSDirectory.open(indexPath);
+    return new IndexWriter(directory, getIndexWriterConfig());
+  }
+
+  protected IndexWriterConfig getIndexWriterConfig() {
+    return defaultIndexWriterConfig();
+  }
+
+  public MonitorQuerySerializer getQuerySerializer() {
+    return serializer;
+  }
+
+  /**
+   * Set the QueryDecomposer to be used by the Monitor
+   *
+   * @param queryDecomposer the QueryDecomposer to be used by the Monitor
+   * @return the current configuration
+   */
+  public MonitorConfiguration setQueryDecomposer(QueryDecomposer queryDecomposer) {
+    this.queryDecomposer = queryDecomposer;
+    return this;
+  }
+
+  /**
+   * @return the QueryDecomposer used by the Monitor
+   */
+  public QueryDecomposer getQueryDecomposer() {
+    return queryDecomposer;
+  }
+
+  /**
+   * Set the frequency with with the Monitor's querycache will be garbage-collected
+   *
+   * @param frequency the frequency value
+   * @param units     the frequency units
+   * @return the current configuration
+   */
+  public MonitorConfiguration setPurgeFrequency(long frequency, TimeUnit units) {
+    this.purgeFrequency = frequency;
+    this.purgeFrequencyUnits = units;
+    return this;
+  }
+
+  /**
+   * @return the value of Monitor's querycache garbage-collection frequency
+   */
+  public long getPurgeFrequency() {
+    return purgeFrequency;
+  }
+
+  /**
+   * @return Get the units of the Monitor's querycache garbage-collection frequency
+   */
+  public TimeUnit getPurgeFrequencyUnits() {
+    return purgeFrequencyUnits;
+  }
+
+  /**
+   * Set how many queries will be buffered in memory before being committed to the queryindex
+   *
+   * @param size how many queries will be buffered in memory before being committed to the queryindex
+   * @return the current configuration
+   */
+  public MonitorConfiguration setQueryUpdateBufferSize(int size) {
+    this.queryUpdateBufferSize = size;
+    return this;
+  }
+
+  /**
+   * @return the size of the queryindex's in-memory buffer
+   */
+  public int getQueryUpdateBufferSize() {
+    return queryUpdateBufferSize;
+  }
+
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/MonitorQuery.java b/lucene/monitor/src/java/org/apache/lucene/monitor/MonitorQuery.java
new file mode 100644
index 0000000..2930817
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/MonitorQuery.java
@@ -0,0 +1,135 @@
+/*
+ * 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.util.Collections;
+import java.util.Map;
+import java.util.Objects;
+import java.util.TreeMap;
+
+import org.apache.lucene.search.Query;
+
+/**
+ * Defines a query to be stored in a Monitor
+ */
+public class MonitorQuery {
+
+  private final String id;
+  private final Query query;
+  private final String queryString;
+  private final Map<String, String> metadata;
+
+  /**
+   * Creates a new MonitorQuery
+   *
+   * @param id          the query ID
+   * @param query       the query to store
+   * @param queryString an optional string representation of the query, for persistent Monitors
+   * @param metadata    metadata passed to {@link Presearcher#indexQuery(Query, Map)}.  Must not
+   *                    have any null values
+   */
+  public MonitorQuery(String id, Query query, String queryString, Map<String, String> metadata) {
+    this.id = id;
+    this.query = query;
+    this.queryString = queryString;
+    this.metadata = Collections.unmodifiableMap(new TreeMap<>(metadata));
+    checkNullEntries(this.metadata);
+  }
+
+  /**
+   * Creates a new MonitorQuery with empty metadata and no string representation
+   *
+   * @param id    the ID
+   * @param query the query
+   */
+  public MonitorQuery(String id, Query query) {
+    this(id, query, null, Collections.emptyMap());
+  }
+
+  private static void checkNullEntries(Map<String, String> metadata) {
+    for (Map.Entry<String, String> entry : metadata.entrySet()) {
+      if (entry.getValue() == null)
+        throw new IllegalArgumentException("Null value for key " + entry.getKey() + " in metadata map");
+    }
+  }
+
+  /**
+   * @return this MonitorQuery's ID
+   */
+  public String getId() {
+    return id;
+  }
+
+  /**
+   * @return this MonitorQuery's query
+   */
+  public Query getQuery() {
+    return query;
+  }
+
+  /**
+   * @return this MonitorQuery's string representation
+   */
+  public String getQueryString() {
+    return queryString;
+  }
+
+  /**
+   * @return this MonitorQuery's metadata
+   */
+  public Map<String, String> getMetadata() {
+    return metadata;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    MonitorQuery that = (MonitorQuery) o;
+    return Objects.equals(id, that.id) && Objects.equals(query, that.query) && Objects.equals(metadata, that.metadata);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(id, query, metadata);
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder(id);
+    sb.append(": ");
+    if (queryString == null) {
+      sb.append(query.toString());
+    }
+    else {
+      sb.append(queryString);
+    }
+    if (metadata.size() != 0) {
+      sb.append(" { ");
+      int n = metadata.size();
+      for (Map.Entry<String, String> entry : metadata.entrySet()) {
+        n--;
+        sb.append(entry.getKey()).append(": ").append(entry.getValue());
+        if (n > 0)
+          sb.append(", ");
+      }
+      sb.append(" }");
+    }
+    return sb.toString();
+  }
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/MonitorQuerySerializer.java b/lucene/monitor/src/java/org/apache/lucene/monitor/MonitorQuerySerializer.java
new file mode 100644
index 0000000..c86dfca
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/MonitorQuerySerializer.java
@@ -0,0 +1,92 @@
+/*
+ * 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.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.function.Function;
+
+import org.apache.lucene.search.Query;
+import org.apache.lucene.store.InputStreamDataInput;
+import org.apache.lucene.store.OutputStreamDataOutput;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Serializes and deserializes MonitorQuery objects into byte streams
+ *
+ * Use this for persistent query indexes
+ */
+public interface MonitorQuerySerializer {
+
+  /**
+   * Builds a MonitorQuery from a byte representation
+   */
+  MonitorQuery deserialize(BytesRef binaryValue);
+
+  /**
+   * Converts a MonitorQuery into a byte representation
+   */
+  BytesRef serialize(MonitorQuery query);
+
+  /**
+   * Build a serializer from a query parser
+   *
+   * @param parser a parser to convert a String representation of a query into a lucene query object
+   */
+  static MonitorQuerySerializer fromParser(Function<String, Query> parser) {
+    return new MonitorQuerySerializer() {
+      @Override
+      public MonitorQuery deserialize(BytesRef binaryValue) {
+        ByteArrayInputStream is = new ByteArrayInputStream(binaryValue.bytes);
+        try (InputStreamDataInput data = new InputStreamDataInput(is)) {
+          String id = data.readString();
+          String query = data.readString();
+          Map<String, String> metadata = new HashMap<>();
+          for (int i = data.readInt(); i > 0; i--) {
+            metadata.put(data.readString(), data.readString());
+          }
+          return new MonitorQuery(id, parser.apply(query), query, metadata);
+        } catch (IOException e) {
+          throw new RuntimeException(e);  // shouldn't happen, we're reading from a bytearray!
+        }
+      }
+
+      @Override
+      public BytesRef serialize(MonitorQuery query) {
+        ByteArrayOutputStream os = new ByteArrayOutputStream();
+        try (OutputStreamDataOutput data = new OutputStreamDataOutput(os)) {
+          data.writeString(query.getId());
+          data.writeString(query.getQueryString());
+          data.writeInt(query.getMetadata().size());
+          for (Map.Entry<String, String> entry : query.getMetadata().entrySet()) {
+            data.writeString(entry.getKey());
+            data.writeString(entry.getValue());
+          }
+          return new BytesRef(os.toByteArray());
+        }
+        catch (IOException e) {
+          throw new RuntimeException(e);  // All in memory, so no IOException should be thrown
+        }
+      }
+    };
+  }
+
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/MonitorUpdateListener.java b/lucene/monitor/src/java/org/apache/lucene/monitor/MonitorUpdateListener.java
new file mode 100644
index 0000000..e3d2612
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/MonitorUpdateListener.java
@@ -0,0 +1,52 @@
+/*
+ * 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.util.List;
+
+/**
+ * For reporting events on a Monitor's query index
+ */
+public interface MonitorUpdateListener {
+
+  /**
+   * Called after a set of queries have been added to the Monitor's query index
+   */
+  default void afterUpdate(List<MonitorQuery> updates) {};
+
+  /**
+   * Called after a set of queries have been deleted from the Monitor's query index
+   */
+  default void afterDelete(List<String> queryIds) {};
+
+  /**
+   * Called after all queries have been removed from the Monitor's query index
+   */
+  default void afterClear() {};
+
+  /**
+   * Called after the Monitor's query cache has been purged of deleted queries
+   */
+  default void onPurge() {};
+
+  /**
+   * Called if there was an error removing deleted queries from the Monitor's query cache
+   */
+  default void onPurgeError(Throwable t) {};
+
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/MultiMatchingQueries.java b/lucene/monitor/src/java/org/apache/lucene/monitor/MultiMatchingQueries.java
new file mode 100644
index 0000000..79cb6b4
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/MultiMatchingQueries.java
@@ -0,0 +1,123 @@
+/*
+ * 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.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Class to hold the results of matching a batch of {@link org.apache.lucene.document.Document}s
+ * against queries held in the Monitor
+ *
+ * @param <T> the type of QueryMatch returned
+ */
+public class MultiMatchingQueries<T extends QueryMatch> {
+
+  private final List<Map<String, T>> matches;
+  private final Map<String, Exception> errors;
+
+  private final long queryBuildTime;
+  private final long searchTime;
+  private final int queriesRun;
+  private final int batchSize;
+
+  MultiMatchingQueries(List<Map<String, T>> matches, Map<String, Exception> errors,
+                  long queryBuildTime, long searchTime, int queriesRun, int batchSize) {
+    this.matches = Collections.unmodifiableList(matches);
+    this.errors = Collections.unmodifiableMap(errors);
+    this.queryBuildTime = queryBuildTime;
+    this.searchTime = searchTime;
+    this.queriesRun = queriesRun;
+    this.batchSize = batchSize;
+  }
+
+  /**
+   * Returns the QueryMatch for the given query and document, or null if it did not match
+   *
+   * @param queryId the query id
+   * @param docId   the doc id
+   * @return the QueryMatch for the given query and document, or null if it did not match
+   */
+  public T matches(String queryId, int docId) {
+    Map<String, T> docMatches = matches.get(docId);
+    if (docMatches == null)
+      return null;
+    return docMatches.get(queryId);
+  }
+
+  /**
+   * @param docId document id to check
+   * @return all matches for a particular document
+   */
+  public Collection<T> getMatches(int docId) {
+    return matches.get(docId).values();
+  }
+
+  /**
+   * @param docId document id to check
+   * @return the number of queries that matched for a given document
+   */
+  public int getMatchCount(int docId) {
+    Map<String, T> docMatches = matches.get(docId);
+    if (docMatches == null)
+      return 0;
+    return docMatches.size();
+  }
+
+  /**
+   * @return how long (in ms) it took to build the Presearcher query for the matcher run
+   */
+  public long getQueryBuildTime() {
+    return queryBuildTime;
+  }
+
+  /**
+   * @return how long (in ms) it took to run the selected queries
+   */
+  public long getSearchTime() {
+    return searchTime;
+  }
+
+  /**
+   * @return the number of queries passed to this CandidateMatcher during the matcher run
+   */
+  public int getQueriesRun() {
+    return queriesRun;
+  }
+
+  /**
+   * @return the number of documents in the batch
+   */
+  public int getBatchSize() {
+    return batchSize;
+  }
+
+  /**
+   * @return a List of any MatchErrors created during the matcher run
+   */
+  public Map<String, Exception> getErrors() {
+    return errors;
+  }
+
+  MatchingQueries<T> singleton() {
+    assert matches.size() == 1;
+    return new MatchingQueries<>(matches.get(0), errors, queryBuildTime, searchTime, queriesRun);
+  }
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/MultipassTermFilteredPresearcher.java b/lucene/monitor/src/java/org/apache/lucene/monitor/MultipassTermFilteredPresearcher.java
new file mode 100644
index 0000000..b155e13
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/MultipassTermFilteredPresearcher.java
@@ -0,0 +1,161 @@
+/*
+ * 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.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermInSetQuery;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefHash;
+
+/**
+ * A TermFilteredPresearcher that indexes queries multiple times, with terms collected
+ * from different routes through a querytree.  Each route will produce a set of terms
+ * that are *sufficient* to select the query, and are indexed into a separate, suffixed field.
+ * <p>
+ * Incoming documents are then converted to a set of Disjunction queries over each
+ * suffixed field, and these queries are combined into a conjunction query, such that the
+ * document's set of terms must match a term from each route.
+ * <p>
+ * This allows filtering out of documents that contain one half of a two-term phrase query, for
+ * example.  The query {@code "hello world"} will be indexed twice, once under 'hello' and once
+ * under 'world'.  A document containing the terms "hello there" would match the first field,
+ * but not the second, and so would not be selected for matching.
+ * <p>
+ * The number of passes the presearcher makes is configurable.  More passes will improve the
+ * selected/matched ratio, but will take longer to index and will use more RAM.
+ * <p>
+ * A minimum weight can we set for terms to be chosen for the second and subsequent passes.  This
+ * allows users to avoid indexing stopwords, for example.
+ */
+public class MultipassTermFilteredPresearcher extends TermFilteredPresearcher {
+
+  private final int passes;
+  private final float minWeight;
+
+  /**
+   * Construct a new MultipassTermFilteredPresearcher
+   *
+   * @param passes        the number of times a query should be indexed
+   * @param minWeight     the minimum weight a querytree should be advanced over
+   * @param weightor      the TreeWeightor to use
+   * @param queryHandlers a list of custom query handlers
+   * @param filterFields  a set of fields to use as filters
+   */
+  public MultipassTermFilteredPresearcher(int passes, float minWeight, TermWeightor weightor,
+                                          List<CustomQueryHandler> queryHandlers, Set<String> filterFields) {
+    super(weightor, queryHandlers, filterFields);
+    this.passes = passes;
+    this.minWeight = minWeight;
+  }
+
+  /**
+   * Construct a new MultipassTermFilteredPresearcher using {@link TermFilteredPresearcher#DEFAULT_WEIGHTOR}
+   * <p>
+   * Note that this will be constructed with a minimum advance weight of zero
+   *
+   * @param passes     the number of times a query should be indexed
+   */
+  public MultipassTermFilteredPresearcher(int passes) {
+    this(passes, 0, DEFAULT_WEIGHTOR, Collections.emptyList(), Collections.emptySet());
+  }
+
+  @Override
+  protected DocumentQueryBuilder getQueryBuilder() {
+    return new MultipassDocumentQueryBuilder();
+  }
+
+  private static String field(String field, int pass) {
+    return field + "_" + pass;
+  }
+
+  private class MultipassDocumentQueryBuilder implements DocumentQueryBuilder {
+
+    BooleanQuery.Builder[] queries = new BooleanQuery.Builder[passes];
+    Map<String, BytesRefHash> terms = new HashMap<>();
+
+    MultipassDocumentQueryBuilder() {
+      for (int i = 0; i < queries.length; i++) {
+        queries[i] = new BooleanQuery.Builder();
+      }
+    }
+
+    @Override
+    public void addTerm(String field, BytesRef term) {
+      BytesRefHash t = terms.computeIfAbsent(field, f -> new BytesRefHash());
+      t.add(term);
+    }
+
+    @Override
+    public Query build() {
+      Map<String, BytesRef[]> collectedTerms = new HashMap<>();
+      for (String field : terms.keySet()) {
+        collectedTerms.put(field, convertHash(terms.get(field)));
+      }
+      BooleanQuery.Builder parent = new BooleanQuery.Builder();
+      for (int i = 0; i < passes; i++) {
+        BooleanQuery.Builder child = new BooleanQuery.Builder();
+        for (String field : terms.keySet()) {
+          child.add(new TermInSetQuery(field(field, i), collectedTerms.get(field)), BooleanClause.Occur.SHOULD);
+        }
+        parent.add(child.build(), BooleanClause.Occur.MUST);
+      }
+      return parent.build();
+    }
+  }
+
+  @Override
+  public Document buildQueryDocument(QueryTree querytree) {
+
+    Document doc = new Document();
+
+    for (int i = 0; i < passes; i++) {
+      Map<String, BytesRefHash> fieldTerms = collectTerms(querytree);
+      for (Map.Entry<String, BytesRefHash> entry : fieldTerms.entrySet()) {
+        // we add the index terms once under a suffixed field for the multipass query, and
+        // once under the plan field name for the TermsEnumTokenFilter
+        doc.add(new Field(field(entry.getKey(), i),
+            new TermsEnumTokenStream(new BytesRefHashIterator(entry.getValue())), QUERYFIELDTYPE));
+        doc.add(new Field(entry.getKey(),
+            new TermsEnumTokenStream(new BytesRefHashIterator(entry.getValue())), QUERYFIELDTYPE));
+      }
+      querytree.advancePhase(minWeight);
+    }
+
+    return doc;
+  }
+
+  private static BytesRef[] convertHash(BytesRefHash hash) {
+    BytesRef[] terms = new BytesRef[hash.size()];
+    for (int i = 0; i < terms.length; i++) {
+      BytesRef t = new BytesRef();
+      terms[i] = hash.get(i, t);
+    }
+    return terms;
+  }
+
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/ParallelMatcher.java b/lucene/monitor/src/java/org/apache/lucene/monitor/ParallelMatcher.java
new file mode 100644
index 0000000..237c9e2
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/ParallelMatcher.java
@@ -0,0 +1,205 @@
+/*
+ * 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.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+
+/**
+ * Matcher class that runs matching queries in parallel.
+ * <p>
+ * This class delegates the actual matching to separate CandidateMatcher classes,
+ * built from a passed in MatcherFactory.
+ * <p>
+ * Use this when individual queries can take a long time to run, and you want
+ * to minimize latency.  The matcher distributes queries amongst its worker
+ * threads using a BlockingQueue, and synchronization overhead may affect performance
+ * if the individual queries are very fast.
+ *
+ * @param <T> the QueryMatch type returned
+ * @see PartitionMatcher
+ */
+public class ParallelMatcher<T extends QueryMatch> extends CandidateMatcher<T> {
+
+  private final BlockingQueue<MatcherTask> queue = new LinkedBlockingQueue<>(1024);
+
+  private final List<Future<CandidateMatcher<T>>> futures = new ArrayList<>();
+
+  private final CandidateMatcher<T> collectorMatcher;
+
+  /**
+   * Create a new ParallelMatcher
+   *
+   * @param searcher       the IndexSearcher to match against
+   * @param executor       an ExecutorService to use for parallel execution
+   * @param matcherFactory MatcherFactory to use to create CandidateMatchers
+   * @param threads        the number of threads to execute on
+   */
+  private ParallelMatcher(IndexSearcher searcher, ExecutorService executor,
+                          MatcherFactory<T> matcherFactory, int threads) {
+    super(searcher);
+    for (int i = 0; i < threads; i++) {
+      MatcherWorker mw = new MatcherWorker(matcherFactory);
+      futures.add(executor.submit(mw));
+    }
+    collectorMatcher = matcherFactory.createMatcher(searcher);
+  }
+
+  @Override
+  protected void matchQuery(String queryId, Query matchQuery, Map<String, String> metadata) throws IOException {
+    try {
+      queue.put(new MatcherTask(queryId, matchQuery, metadata));
+    } catch (InterruptedException e) {
+      throw new IOException("Interrupted during match", e);
+    }
+  }
+
+  @Override
+  public T resolve(T match1, T match2) {
+    return collectorMatcher.resolve(match1, match2);
+  }
+
+  @Override
+  protected void doFinish() {
+    try {
+      for (int i = 0; i < futures.size(); i++) {
+        queue.put(END);
+      }
+
+      for (Future<CandidateMatcher<T>> future : futures) {
+        MultiMatchingQueries<T> matches = future.get().finish(0, 0);
+        for (int doc = 0; doc < matches.getBatchSize(); doc++) {
+          for (T match : matches.getMatches(doc)) {
+            this.addMatch(match, doc);
+          }
+        }
+        for (Map.Entry<String, Exception> error : matches.getErrors().entrySet()) {
+          this.reportError(error.getKey(), error.getValue());
+        }
+      }
+
+    } catch (InterruptedException | ExecutionException e) {
+      throw new RuntimeException("Interrupted during match", e);
+    }
+  }
+
+  private class MatcherWorker implements Callable<CandidateMatcher<T>> {
+
+    final CandidateMatcher<T> matcher;
+
+    private MatcherWorker(MatcherFactory<T> matcherFactory) {
+      this.matcher = matcherFactory.createMatcher(searcher);
+    }
+
+    @Override
+    public CandidateMatcher<T> call() {
+      MatcherTask task;
+      try {
+        while ((task = queue.take()) != END) {
+          try {
+            matcher.matchQuery(task.id, task.matchQuery, task.metadata);
+          } catch (IOException e) {
+            matcher.reportError(task.id, e);
+          }
+        }
+      } catch (InterruptedException e) {
+        throw new RuntimeException("Interrupted during match", e);
+      }
+      return matcher;
+    }
+
+  }
+
+  private static class MatcherTask {
+
+    final String id;
+    final Query matchQuery;
+    final Map<String, String> metadata;
+
+    private MatcherTask(String id, Query matchQuery, Map<String, String> metadata) {
+      this.id = id;
+      this.matchQuery = matchQuery;
+      this.metadata = metadata;
+    }
+  }
+
+  /* Marker object placed on the queue after all matches are done, to indicate to the
+     worker threads that they should finish */
+  private static final MatcherTask END = new MatcherTask("", null, Collections.emptyMap());
+
+  private static class ParallelMatcherFactory<T extends QueryMatch> implements MatcherFactory<T> {
+
+    private final ExecutorService executor;
+    private final MatcherFactory<T> matcherFactory;
+    private final int threads;
+
+    ParallelMatcherFactory(ExecutorService executor, MatcherFactory<T> matcherFactory,
+                                  int threads) {
+      this.executor = executor;
+      this.matcherFactory = matcherFactory;
+      this.threads = threads;
+    }
+
+    @Override
+    public ParallelMatcher<T> createMatcher(IndexSearcher searcher) {
+      return new ParallelMatcher<>(searcher, executor, matcherFactory, threads);
+    }
+  }
+
+  /**
+   * Create a new MatcherFactory for a ParallelMatcher
+   *
+   * @param executor       the ExecutorService to use
+   * @param matcherFactory the MatcherFactory to use to create submatchers
+   * @param threads        the number of threads to use
+   * @param <T>            the type of QueryMatch generated
+   */
+  public static <T extends QueryMatch> MatcherFactory<T> factory(ExecutorService executor,
+                                                                         MatcherFactory<T> matcherFactory, int threads) {
+    return new ParallelMatcherFactory<>(executor, matcherFactory, threads);
+  }
+
+  /**
+   * Create a new MatcherFactory for a ParallelMatcher
+   * <p>
+   * This factory will create a ParallelMatcher that uses as many threads as there are cores available
+   * to the JVM (as determined by {@code Runtime.getRuntime().availableProcessors()}).
+   *
+   * @param executor       the ExecutorService to use
+   * @param matcherFactory the MatcherFactory to use to create submatchers
+   * @param <T>            the type of QueryMatch generated
+   */
+  public static <T extends QueryMatch> MatcherFactory<T> factory(ExecutorService executor,
+                                                                         MatcherFactory<T> matcherFactory) {
+    int threads = Runtime.getRuntime().availableProcessors();
+    return new ParallelMatcherFactory<>(executor, matcherFactory, threads);
+  }
+
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/PartitionMatcher.java b/lucene/monitor/src/java/org/apache/lucene/monitor/PartitionMatcher.java
new file mode 100644
index 0000000..42857c7
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/PartitionMatcher.java
@@ -0,0 +1,202 @@
+/*
+ * 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.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+
+/**
+ * A multi-threaded matcher that collects all possible matches in one pass, and
+ * then partitions them amongst a number of worker threads to perform the actual
+ * matching.
+ * <p>
+ * This class delegates the matching to separate CandidateMatcher classes,
+ * built from a passed in MatcherFactory.
+ * <p>
+ * Use this if your query sets contain large numbers of very fast queries, where
+ * the synchronization overhead of {@link ParallelMatcher}
+ * can outweigh the benefit of multithreading.
+ *
+ * @param <T> the type of QueryMatch to return
+ * @see ParallelMatcher
+ */
+public class PartitionMatcher<T extends QueryMatch> extends CandidateMatcher<T> {
+
+  private final ExecutorService executor;
+
+  private final MatcherFactory<T> matcherFactory;
+
+  private final int threads;
+
+  private final CandidateMatcher<T> resolvingMatcher;
+
+  private static class MatchTask {
+
+    final String queryId;
+    final Query matchQuery;
+    final Map<String, String> metadata;
+
+    private MatchTask(String queryId, Query matchQuery, Map<String, String> metadata) {
+      this.queryId = queryId;
+      this.matchQuery = matchQuery;
+      this.metadata = metadata;
+    }
+  }
+
+  private final List<MatchTask> tasks = new ArrayList<>();
+
+  private PartitionMatcher(IndexSearcher searcher, ExecutorService executor, MatcherFactory<T> matcherFactory, int threads) {
+    super(searcher);
+    this.executor = executor;
+    this.matcherFactory = matcherFactory;
+    this.threads = threads;
+    this.resolvingMatcher = matcherFactory.createMatcher(searcher);
+  }
+
+  @Override
+  protected void matchQuery(String queryId, Query matchQuery, Map<String, String> metadata) {
+    tasks.add(new MatchTask(queryId, matchQuery, metadata));
+  }
+
+  @Override
+  public T resolve(T match1, T match2) {
+    return resolvingMatcher.resolve(match1, match2);
+  }
+
+  @Override
+  protected void doFinish() {
+
+    List<Callable<MultiMatchingQueries<T>>> workers = new ArrayList<>(threads);
+    for (List<MatchTask> taskset : partition(tasks, threads)) {
+      CandidateMatcher<T> matcher = matcherFactory.createMatcher(searcher);
+      workers.add(new MatcherWorker(taskset, matcher));
+    }
+
+    try {
+      for (Future<MultiMatchingQueries<T>> future : executor.invokeAll(workers)) {
+        MultiMatchingQueries<T> matches = future.get();
+        for (int doc = 0; doc < matches.getBatchSize(); doc++) {
+          for (T match : matches.getMatches(doc)) {
+            addMatch(match, doc);
+          }
+        }
+      }
+
+    } catch (InterruptedException | ExecutionException e) {
+      throw new RuntimeException("Interrupted during match", e);
+    }
+
+  }
+
+  private class MatcherWorker implements Callable<MultiMatchingQueries<T>> {
+
+    final List<MatchTask> tasks;
+    final CandidateMatcher<T> matcher;
+
+    private MatcherWorker(List<MatchTask> tasks, CandidateMatcher<T> matcher) {
+      this.tasks = tasks;
+      this.matcher = matcher;
+    }
+
+    @Override
+    public MultiMatchingQueries<T> call() {
+      for (MatchTask task : tasks) {
+        try {
+          matcher.matchQuery(task.queryId, task.matchQuery, task.metadata);
+        } catch (IOException e) {
+          PartitionMatcher.this.reportError(task.queryId, e);
+        }
+      }
+      return matcher.finish(0, 0);
+    }
+  }
+
+  private static class PartitionMatcherFactory<T extends QueryMatch> implements MatcherFactory<T> {
+
+    private final ExecutorService executor;
+    private final MatcherFactory<T> matcherFactory;
+    private final int threads;
+
+    PartitionMatcherFactory(ExecutorService executor, MatcherFactory<T> matcherFactory,
+                                   int threads) {
+      this.executor = executor;
+      this.matcherFactory = matcherFactory;
+      this.threads = threads;
+    }
+
+    @Override
+    public PartitionMatcher<T> createMatcher(IndexSearcher searcher) {
+      return new PartitionMatcher<>(searcher, executor, matcherFactory, threads);
+    }
+  }
+
+  /**
+   * Create a new MatcherFactory for a PartitionMatcher
+   *
+   * @param executor       the ExecutorService to use
+   * @param matcherFactory the MatcherFactory to use to create submatchers
+   * @param threads        the number of threads to use
+   * @param <T>            the type of QueryMatch generated
+   */
+  public static <T extends QueryMatch> MatcherFactory<T> factory(ExecutorService executor,
+                                                                          MatcherFactory<T> matcherFactory, int threads) {
+    return new PartitionMatcherFactory<>(executor, matcherFactory, threads);
+  }
+
+  /**
+   * Create a new MatcherFactory for a PartitionMatcher
+   * <p>
+   * This factory will create a PartitionMatcher that uses as many threads as there are cores available
+   * to the JVM (as determined by {@code Runtime.getRuntime().availableProcessors()}).
+   *
+   * @param executor       the ExecutorService to use
+   * @param matcherFactory the MatcherFactory to use to create submatchers
+   * @param <T>            the type of QueryMatch generated
+   */
+  public static <T extends QueryMatch> MatcherFactory<T> factory(ExecutorService executor,
+                                                                          MatcherFactory<T> matcherFactory) {
+    int threads = Runtime.getRuntime().availableProcessors();
+    return new PartitionMatcherFactory<>(executor, matcherFactory, threads);
+  }
+
+  static <T> List<List<T>> partition(List<T> items, int slices) {
+    double size = items.size() / (double) slices;
+    double accum = 0;
+    int start = 0;
+    List<List<T>> list = new ArrayList<>(slices);
+    for (int i = 0; i < slices; i++) {
+      int end = (int) Math.floor(accum + size);
+      if (i == slices - 1)
+        end = items.size();
+      list.add(items.subList(start, end));
+      accum += size;
+      start = (int) Math.floor(accum);
+    }
+    return list;
+  }
+
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/Presearcher.java b/lucene/monitor/src/java/org/apache/lucene/monitor/Presearcher.java
new file mode 100644
index 0000000..bc5a5f1
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/Presearcher.java
@@ -0,0 +1,71 @@
+/*
+ * 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.util.Map;
+import java.util.function.BiPredicate;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A Presearcher is used by the Monitor to reduce the number of queries actually
+ * run against a Document.  It defines how queries are stored in the monitor's
+ * internal index, and how a Document is converted to a query against that
+ * index.
+ */
+public abstract class Presearcher {
+
+  /**
+   * A Presearcher implementation that does no query filtering, and runs all
+   * registered queries
+   */
+  public static final Presearcher NO_FILTERING = new Presearcher() {
+    @Override
+    public Query buildQuery(LeafReader reader, BiPredicate<String, BytesRef> termAcceptor) {
+      return new MatchAllDocsQuery();
+    }
+
+    @Override
+    public Document indexQuery(Query query, Map<String, String> metadata) {
+      return new Document();
+    }
+  };
+
+  /**
+   * Build a query for a Monitor's queryindex from a LeafReader over a set of documents to monitor.
+   *
+   * @param reader          a {@link LeafReader} over the input documents
+   * @param termAcceptor    a predicate indicating if a term should be added to the query
+   * @return a Query to run over a Monitor's queryindex
+   */
+  public abstract Query buildQuery(LeafReader reader, BiPredicate<String, BytesRef> termAcceptor);
+
+  /**
+   * Build a lucene Document to index the query in a Monitor's queryindex
+   *
+   * @param query    the Query to index
+   * @param metadata a Map of arbitrary query metadata
+   * @return a lucene Document to add to the queryindex
+   */
+  public abstract Document indexQuery(Query query, Map<String, String> metadata);
+
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/PresearcherMatch.java b/lucene/monitor/src/java/org/apache/lucene/monitor/PresearcherMatch.java
new file mode 100644
index 0000000..c74988b
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/PresearcherMatch.java
@@ -0,0 +1,45 @@
+/*
+ * 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;
+
+/**
+ * Wraps a {@link QueryMatch} with information about which queries were selected by the presearcher
+ */
+public class PresearcherMatch<T extends QueryMatch> {
+
+  /**
+   * The presearcher hits
+   */
+  public final String presearcherMatches;
+
+  /**
+   * The QueryMatch
+   */
+  public final T queryMatch;
+
+  /**
+   * The query id
+   */
+  public final String queryId;
+
+  PresearcherMatch(String id, String presearcherMatches, T queryMatch) {
+    this.presearcherMatches = presearcherMatches;
+    this.queryMatch = queryMatch;
+    this.queryId = id;
+  }
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/PresearcherMatches.java b/lucene/monitor/src/java/org/apache/lucene/monitor/PresearcherMatches.java
new file mode 100644
index 0000000..464cf88
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/PresearcherMatches.java
@@ -0,0 +1,50 @@
+/*
+ * 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.util.Map;
+
+/**
+ * Wraps a {@link MultiMatchingQueries} with information on which presearcher queries were selected
+ */
+public class PresearcherMatches<T extends QueryMatch> {
+
+  private final Map<String, StringBuilder> matchingTerms;
+
+  /** The wrapped Matches */
+  public final MultiMatchingQueries<T> matcher;
+
+  /**
+   * Builds a new PresearcherMatches
+   */
+  public PresearcherMatches(Map<String, StringBuilder> matchingTerms, MultiMatchingQueries<T> matcher) {
+    this.matcher = matcher;
+    this.matchingTerms = matchingTerms;
+  }
+
+  /**
+   * Returns match information for a given query
+   */
+  public PresearcherMatch<T> match(String queryId, int doc) {
+    StringBuilder found = matchingTerms.get(queryId);
+    if (found != null)
+      return new PresearcherMatch<>(queryId, found.toString(), matcher.matches(queryId, doc));
+    return null;
+  }
+
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/QueryAnalyzer.java b/lucene/monitor/src/java/org/apache/lucene/monitor/QueryAnalyzer.java
new file mode 100644
index 0000000..91b7a08
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/QueryAnalyzer.java
@@ -0,0 +1,143 @@
+/*
+ * 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.util.ArrayList;
+import java.util.List;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+
+/**
+ * Class to analyze and extract terms from a lucene query, to be used by
+ * a {@link Presearcher} in indexing.
+ */
+class QueryAnalyzer {
+
+  private final BiFunction<Query, TermWeightor, QueryTree> unknownQueryMapper;
+
+  QueryAnalyzer(List<CustomQueryHandler> queryTreeBuilders) {
+    this.unknownQueryMapper = buildMapper(queryTreeBuilders);
+  }
+
+  QueryAnalyzer() {
+    this.unknownQueryMapper = (q, w) -> null;
+  }
+
+  private static BiFunction<Query, TermWeightor, QueryTree> buildMapper(List<CustomQueryHandler> mappers) {
+    return (q, w) -> {
+      for (CustomQueryHandler mapper : mappers) {
+        QueryTree qt = mapper.handleQuery(q, w);
+        if (qt != null) {
+          return qt;
+        }
+      }
+      return null;
+    };
+  }
+
+  /**
+   * Create a {@link QueryTree} from a passed in Query or Filter
+   *
+   * @param luceneQuery the query to analyze
+   * @return a QueryTree describing the analyzed query
+   */
+  QueryTree buildTree(Query luceneQuery, TermWeightor weightor) {
+    QueryBuilder builder = new QueryBuilder();
+    luceneQuery.visit(builder);
+    return builder.apply(weightor);
+  }
+
+  private class QueryBuilder extends QueryVisitor implements Function<TermWeightor, QueryTree> {
+
+    final List<Function<TermWeightor, QueryTree>> children = new ArrayList<>();
+
+    @Override
+    public QueryVisitor getSubVisitor(BooleanClause.Occur occur, Query parent) {
+      if (occur == BooleanClause.Occur.MUST || occur == BooleanClause.Occur.FILTER) {
+        QueryBuilder n = new QueryBuilder();
+        children.add(n);
+        return n;
+      }
+      if (occur == BooleanClause.Occur.MUST_NOT) {
+        // Check if we're in a pure negative disjunction
+        if (parent instanceof BooleanQuery) {
+          BooleanQuery bq = (BooleanQuery) parent;
+          long positiveCount = bq.clauses().stream()
+              .filter(c -> c.getOccur() != BooleanClause.Occur.MUST_NOT)
+              .count();
+          if (positiveCount == 0) {
+            children.add(w -> QueryTree.anyTerm("PURE NEGATIVE QUERY[" + parent + "]"));
+          }
+        }
+        return QueryVisitor.EMPTY_VISITOR;
+      }
+      // It's a disjunction clause.  If the parent has MUST or FILTER clauses, we can
+      // ignore it
+      if (parent instanceof BooleanQuery) {
+        BooleanQuery bq = (BooleanQuery) parent;
+        long requiredCount = bq.clauses().stream()
+            .filter(c -> c.getOccur() == BooleanClause.Occur.MUST || c.getOccur() == BooleanClause.Occur.FILTER)
+            .count();
+        if (requiredCount > 0) {
+          return QueryVisitor.EMPTY_VISITOR;
+        }
+      }
+      Disjunction n = new Disjunction();
+      children.add(n);
+      return n;
+    }
+
+    @Override
+    public void consumeTerms(Query query, Term... terms) {
+      for (Term term : terms) {
+        children.add(w -> QueryTree.term(term, w));
+      }
+    }
+
+    @Override
+    public void visitLeaf(Query query) {
+      children.add(w -> {
+        QueryTree q = unknownQueryMapper.apply(query, w);
+        if (q == null) {
+          return QueryTree.anyTerm(query.toString());
+        }
+        return q;
+      });
+    }
+
+    @Override
+    public QueryTree apply(TermWeightor termWeightor) {
+      return QueryTree.conjunction(children, termWeightor);
+    }
+  }
+
+  private class Disjunction extends QueryBuilder {
+
+    @Override
+    public QueryTree apply(TermWeightor termWeightor) {
+      return QueryTree.disjunction(children, termWeightor);
+    }
+  }
+
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/QueryCacheEntry.java b/lucene/monitor/src/java/org/apache/lucene/monitor/QueryCacheEntry.java
new file mode 100644
index 0000000..cec2784
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/QueryCacheEntry.java
@@ -0,0 +1,71 @@
+/*
+ * 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.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.lucene.search.Query;
+
+class QueryCacheEntry {
+
+  /**
+   * The (possibly partial due to decomposition) query
+   */
+  final Query matchQuery;
+
+  /**
+   * The id of this query
+   */
+  final String cacheId;
+
+  /**
+   * The id of the MonitorQuery that produced this entry
+   *
+   * Note that this may be different to {@link #cacheId} due to decomposition
+   */
+  final String queryId;
+
+  /**
+   * The metadata from the entry's parent {@link MonitorQuery}
+   */
+  final Map<String, String> metadata;
+
+  private QueryCacheEntry(String cacheId, String queryId, Query matchQuery, Map<String, String> metadata) {
+    this.cacheId = cacheId;
+    this.queryId = queryId;
+    this.matchQuery = matchQuery;
+    this.metadata = metadata;
+  }
+
+  static List<QueryCacheEntry> decompose(MonitorQuery mq, QueryDecomposer decomposer) {
+    int upto = 0;
+    List<QueryCacheEntry> cacheEntries = new ArrayList<>();
+    for (Query subquery : decomposer.decompose(mq.getQuery())) {
+      cacheEntries.add(new QueryCacheEntry(mq.getId() + "_" + upto, mq.getId(), subquery, mq.getMetadata()));
+      upto++;
+    }
+    return cacheEntries;
+  }
+
+  @Override
+  public String toString() {
+    return queryId + "/" + cacheId + "/" + matchQuery;
+  }
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/QueryDecomposer.java b/lucene/monitor/src/java/org/apache/lucene/monitor/QueryDecomposer.java
new file mode 100644
index 0000000..93053f6
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/QueryDecomposer.java
@@ -0,0 +1,123 @@
+/*
+ * 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.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.BoostQuery;
+import org.apache.lucene.search.DisjunctionMaxQuery;
+import org.apache.lucene.search.Query;
+
+/**
+ * Split a disjunction query into its consituent parts, so that they can be indexed
+ * and run separately in the Monitor.
+ */
+public class QueryDecomposer {
+
+  /**
+   * Split a query up into individual parts that can be indexed and run separately
+   *
+   * @param q the query
+   * @return a collection of subqueries
+   */
+  public Set<Query> decompose(Query q) {
+
+    if (q instanceof BooleanQuery)
+      return decomposeBoolean((BooleanQuery) q);
+
+    if (q instanceof DisjunctionMaxQuery) {
+      Set<Query> subqueries = new HashSet<>();
+      for (Query subq : ((DisjunctionMaxQuery) q).getDisjuncts()) {
+        subqueries.addAll(decompose(subq));
+      }
+      return subqueries;
+    }
+
+    if (q instanceof BoostQuery) {
+      return decomposeBoostQuery((BoostQuery) q);
+    }
+
+    return Collections.singleton(q);
+  }
+
+  public Set<Query> decomposeBoostQuery(BoostQuery q) {
+    if (q.getBoost() == 1.0)
+      return decompose(q.getQuery());
+
+    Set<Query> boostedDecomposedQueries = new HashSet<>();
+    for (Query subq : decompose(q.getQuery())) {
+      boostedDecomposedQueries.add(new BoostQuery(subq, q.getBoost()));
+    }
+    return boostedDecomposedQueries;
+  }
+
+  /**
+   * Decompose a {@link org.apache.lucene.search.BooleanQuery}
+   *
+   * @param q the boolean query
+   * @return a collection of subqueries
+   */
+  public Set<Query> decomposeBoolean(BooleanQuery q) {
+    if (q.getMinimumNumberShouldMatch() > 1)
+      return Collections.singleton(q);
+
+    Set<Query> subqueries = new HashSet<>();
+    Set<Query> exclusions = new HashSet<>();
+    Set<Query> mandatory = new HashSet<>();
+
+    for (BooleanClause clause : q) {
+      if (clause.getOccur() == BooleanClause.Occur.MUST || clause.getOccur() == BooleanClause.Occur.FILTER)
+        mandatory.add(clause.getQuery());
+      else if (clause.getOccur() == BooleanClause.Occur.MUST_NOT)
+        exclusions.add(clause.getQuery());
+      else {
+        subqueries.addAll(decompose(clause.getQuery()));
+      }
+    }
+
+    // More than one MUST clause, or a single MUST clause with disjunctions
+    if (mandatory.size() > 1 || (mandatory.size() == 1 && subqueries.size() > 0))
+      return Collections.singleton(q);
+
+    // If we only have a single MUST clause and no SHOULD clauses, then we can
+    // decompose the MUST clause instead
+    if (mandatory.size() == 1) {
+      subqueries.addAll(decompose(mandatory.iterator().next()));
+    }
+
+    if (exclusions.size() == 0)
+      return subqueries;
+
+    // If there are exclusions, then we need to add them to all the decomposed
+    // queries
+    Set<Query> rewrittenSubqueries = new HashSet<>(subqueries.size());
+    for (Query subquery : subqueries) {
+      BooleanQuery.Builder bq = new BooleanQuery.Builder();
+      bq.add(subquery, BooleanClause.Occur.MUST);
+      for (Query ex : exclusions) {
+        bq.add(ex, BooleanClause.Occur.MUST_NOT);
+      }
+      rewrittenSubqueries.add(bq.build());
+    }
+    return rewrittenSubqueries;
+  }
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/QueryIndex.java b/lucene/monitor/src/java/org/apache/lucene/monitor/QueryIndex.java
new file mode 100644
index 0000000..9a58f58
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/QueryIndex.java
@@ -0,0 +1,449 @@
+/*
+ * 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.Closeable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.BiPredicate;
+
+import org.apache.lucene.document.BinaryDocValuesField;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.BinaryDocValues;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.SortedDocValues;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorable;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.SearcherFactory;
+import org.apache.lucene.search.SearcherManager;
+import org.apache.lucene.search.SimpleCollector;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefHash;
+import org.apache.lucene.util.IOUtils;
+
+class QueryIndex implements Closeable {
+
+  static final class FIELDS {
+    static final String query_id = "_query_id";
+    static final String cache_id = "_cache_id";
+    static final String mq = "_mq";
+  }
+
+  private final IndexWriter writer;
+  private final SearcherManager manager;
+  private final QueryDecomposer decomposer;
+  private final MonitorQuerySerializer serializer;
+  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();
+
+  /* The current query cache */
+  private volatile ConcurrentMap<String, QueryCacheEntry> queries = new ConcurrentHashMap<>();
+  // NB this is not final because it can be replaced by purgeCache()
+
+  // package-private for testing
+  final Map<IndexReader.CacheKey, QueryTermFilter> termFilters = new HashMap<>();
+
+  QueryIndex(MonitorConfiguration config, Presearcher presearcher) throws IOException {
+    this.writer = config.buildIndexWriter();
+    this.manager = new SearcherManager(writer, true, true, new TermsHashBuilder());
+    this.decomposer = config.getQueryDecomposer();
+    this.serializer = config.getQuerySerializer();
+    this.presearcher = presearcher;
+    populateQueryCache(serializer, decomposer);
+  }
+
+  private void populateQueryCache(MonitorQuerySerializer serializer, QueryDecomposer decomposer) throws IOException {
+    if (serializer == null) {
+      // No query serialization happening here - check that the cache is empty
+      IndexSearcher searcher = manager.acquire();
+      try {
+        if (searcher.count(new MatchAllDocsQuery()) != 0) {
+          throw new IllegalStateException("Attempting to open a non-empty monitor query index with no MonitorQuerySerializer");
+        }
+      }
+      finally {
+        manager.release(searcher);
+      }
+      return;
+    }
+    Set<String> ids = new HashSet<>();
+    List<Exception> errors = new ArrayList<>();
+    purgeCache(newCache -> scan((id, cacheEntry, dataValues) -> {
+      if (ids.contains(id)) {
+        // this is a branch of a query that has already been reconstructed, but
+        // then split by decomposition - we don't need to parse it again
+        return;
+      }
+      ids.add(id);
+      try {
+        MonitorQuery mq = serializer.deserialize(dataValues.mq.binaryValue());
+        for (QueryCacheEntry entry : QueryCacheEntry.decompose(mq, decomposer)) {
+          newCache.put(entry.cacheId, entry);
+        }
+      }
+      catch (Exception e) {
+        errors.add(e);
+      }
+    }));
+    if (errors.size() > 0) {
+      IllegalStateException e = new IllegalStateException("Couldn't parse some queries from the index");
+      for (Exception parseError : errors) {
+        e.addSuppressed(parseError);
+      }
+      throw e;
+    }
+  }
+
+  private class TermsHashBuilder extends SearcherFactory {
+    @Override
+    public IndexSearcher newSearcher(IndexReader reader, IndexReader previousReader) throws IOException {
+      IndexSearcher searcher = super.newSearcher(reader, previousReader);
+      searcher.setQueryCache(null);
+      termFilters.put(reader.getReaderCacheHelper().getKey(), new QueryTermFilter(reader));
+      reader.getReaderCacheHelper().addClosedListener(termFilters::remove);
+      return searcher;
+    }
+  }
+
+  void commit(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;
+  }
+
+  interface QueryBuilder {
+    Query buildQuery(BiPredicate<String, BytesRef> termAcceptor) throws IOException;
+  }
+
+  static class QueryTermFilter implements BiPredicate<String, BytesRef> {
+
+    private final Map<String, BytesRefHash> termsHash = new HashMap<>();
+
+    QueryTermFilter(IndexReader reader) throws IOException {
+      for (LeafReaderContext ctx : reader.leaves()) {
+        for (FieldInfo fi : ctx.reader().getFieldInfos()) {
+          BytesRefHash terms = termsHash.computeIfAbsent(fi.name, f -> new BytesRefHash());
+          Terms t = ctx.reader().terms(fi.name);
+          if (t != null) {
+            TermsEnum te = t.iterator();
+            BytesRef term;
+            while ((term = te.next()) != null) {
+              terms.add(term);
+            }
+          }
+        }
+      }
+    }
+
+    @Override
+    public boolean test(String field, BytesRef term) {
+      BytesRefHash bytes = termsHash.get(field);
+      if (bytes == null) {
+        return false;
+      }
+      return bytes.find(term) != -1;
+    }
+  }
+
+  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(FIELDS.query_id, queryId)),
+        (id, query, dataValues) -> bytesHolder[0] = dataValues.mq.binaryValue());
+    return serializer.deserialize(bytesHolder[0]);
+  }
+
+  void scan(QueryCollector matcher) throws IOException {
+    search(new MatchAllDocsQuery(), matcher);
+  }
+
+  long search(final Query query, QueryCollector matcher) throws IOException {
+    QueryBuilder builder = termFilter -> query;
+    return search(builder, matcher);
+  }
+
+  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();
+      }
+
+      MonitorQueryCollector collector = new MonitorQueryCollector(queries, matcher);
+      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);
+      }
+    }
+  }
+
+  interface CachePopulator {
+    void populateCacheWithIndex(Map<String, QueryCacheEntry> newCache) throws IOException;
+  }
+
+  void purgeCache() throws IOException {
+    purgeCache(newCache -> scan((id, query, dataValues) -> {
+      if (query != null)
+        newCache.put(query.cacheId, query);
+    }));
+  }
+
+  /**
+   * 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
+   */
+  private 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());
+  }
+
+  int numDocs() {
+    return writer.getDocStats().numDocs;
+  }
+
+  int cacheSize() {
+    return queries.size();
+  }
+
+  void deleteQueries(Iterable<String> ids) throws IOException {
+    for (String id : ids) {
+      writer.deleteDocuments(new Term(FIELDS.query_id, id));
+    }
+    commit(Collections.emptyList());
+  }
+
+  void clear() throws IOException {
+    writer.deleteAll();
+    commit(Collections.emptyList());
+  }
+
+  interface QueryCollector {
+
+    void matchQuery(String id, QueryCacheEntry query, DataValues dataValues) throws IOException;
+
+    default ScoreMode scoreMode() {
+      return ScoreMode.COMPLETE_NO_SCORES;
+    }
+
+  }
+
+  // ---------------------------------------------
+  //  Helper classes...
+  // ---------------------------------------------
+
+  static final class DataValues {
+    SortedDocValues queryId;
+    SortedDocValues cacheId;
+    BinaryDocValues mq;
+    Scorable scorer;
+    LeafReaderContext ctx;
+
+    void advanceTo(int doc) throws IOException {
+      assert scorer.docID() == doc;
+      queryId.advanceExact(doc);
+      cacheId.advanceExact(doc);
+      if (mq != null) {
+        mq.advanceExact(doc);
+      }
+    }
+  }
+
+  /**
+   * A Collector that decodes the stored query for each document hit.
+   */
+  static final class MonitorQueryCollector extends SimpleCollector {
+
+    private final Map<String, QueryCacheEntry> queries;
+    private final QueryCollector matcher;
+    private final DataValues dataValues = new DataValues();
+
+    MonitorQueryCollector(Map<String, QueryCacheEntry> queries, QueryCollector matcher) {
+      this.queries = queries;
+      this.matcher = matcher;
+    }
+
+    @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.binaryValue();
+      BytesRef query_id = dataValues.queryId.binaryValue();
+      QueryCacheEntry query = queries.get(cache_id.utf8ToString());
+      matcher.matchQuery(query_id.utf8ToString(), query, dataValues);
+    }
+
+    @Override
+    public void doSetNextReader(LeafReaderContext context) throws IOException {
+      this.dataValues.cacheId = context.reader().getSortedDocValues(FIELDS.cache_id);
+      this.dataValues.queryId = context.reader().getSortedDocValues(FIELDS.query_id);
+      this.dataValues.mq = context.reader().getBinaryDocValues(FIELDS.mq);
+      this.dataValues.ctx = context;
+    }
+
+    @Override
+    public ScoreMode scoreMode() {
+      return matcher.scoreMode();
+    }
+
+  }
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/QueryMatch.java b/lucene/monitor/src/java/org/apache/lucene/monitor/QueryMatch.java
new file mode 100644
index 0000000..8290213
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/QueryMatch.java
@@ -0,0 +1,84 @@
+/*
+ * 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.util.Objects;
+
+import org.apache.lucene.search.Scorable;
+import org.apache.lucene.search.ScoreMode;
+
+/**
+ * Represents a match for a specific query and document
+ * <p>
+ * Derived classes may contain more information (such as scores, highlights, etc)
+ *
+ * @see ExplainingMatch
+ * @see ScoringMatch
+ * @see HighlightsMatch
+ */
+public class QueryMatch {
+
+  private final String queryId;
+
+  public static final MatcherFactory<QueryMatch> SIMPLE_MATCHER =
+      searcher -> new CollectingMatcher<QueryMatch>(searcher, ScoreMode.COMPLETE_NO_SCORES) {
+    @Override
+    public QueryMatch resolve(QueryMatch match1, QueryMatch match2) {
+      return match1;
+    }
+
+    @Override
+    protected QueryMatch doMatch(String queryId, int doc, Scorable scorer) {
+      return new QueryMatch(queryId);
+    }
+  };
+
+  /**
+   * Creates a new QueryMatch for a specific query and document
+   *
+   * @param queryId the query id
+   */
+  public QueryMatch(String queryId) {
+    this.queryId = Objects.requireNonNull(queryId);
+  }
+
+  /**
+   * @return the queryid of this match
+   */
+  public String getQueryId() {
+    return queryId;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (!(o instanceof QueryMatch)) return false;
+    QueryMatch that = (QueryMatch) o;
+    return Objects.equals(queryId, that.queryId);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(queryId);
+  }
+
+  @Override
+  public String toString() {
+    return "Match(query=" + queryId + ")";
+  }
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/QueryTimeListener.java b/lucene/monitor/src/java/org/apache/lucene/monitor/QueryTimeListener.java
new file mode 100644
index 0000000..c198c8f
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/QueryTimeListener.java
@@ -0,0 +1,64 @@
+/*
+ * 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.Map;
+
+import org.apache.lucene.search.Query;
+
+/**
+ * Notified of the time it takes to run individual queries against a set of documents
+ */
+public interface QueryTimeListener {
+
+  /**
+   * How long it took to run a particular query
+   */
+  void logQueryTime(String queryId, long timeInNanos);
+
+  /**
+   * A wrapping matcher factory to log query times to a QueryTimeListener
+   * @param factory   a matcher factory to use for the actual matching
+   * @param listener  the QueryTimeListener
+   */
+  static <T extends QueryMatch> MatcherFactory<T> timingMatcher(MatcherFactory<T> factory, QueryTimeListener listener) {
+    return searcher -> {
+      CandidateMatcher<T> matcher = factory.createMatcher(searcher);
+      return new CandidateMatcher<T>(searcher) {
+        @Override
+        protected void matchQuery(String queryId, Query matchQuery, Map<String, String> metadata) throws IOException {
+          long t = System.nanoTime();
+          matcher.matchQuery(queryId, matchQuery, metadata);
+          t = System.nanoTime() - t;
+          listener.logQueryTime(queryId, t);
+        }
+
+        @Override
+        public T resolve(T match1, T match2) {
+          return matcher.resolve(match1, match2);
+        }
+
+        @Override
+        protected void doFinish() {
+          copyMatches(matcher);
+        }
+      };
+    };
+  }
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/QueryTree.java b/lucene/monitor/src/java/org/apache/lucene/monitor/QueryTree.java
new file mode 100644
index 0000000..cfae8ec
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/QueryTree.java
@@ -0,0 +1,299 @@
+/*
+ * 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.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Optional;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A representation of a node in a query tree
+ *
+ * Queries are analyzed and converted into an abstract tree, consisting
+ * of conjunction and disjunction nodes, and leaf nodes containing terms.
+ *
+ * Terms may be collected from a node, which will use the weights of its
+ * sub-nodes to determine which paths are followed.  The path may be changed
+ * by calling {@link #advancePhase(double)}
+ */
+public abstract class QueryTree {
+
+  /**
+   * The weight of this node
+   */
+  public abstract double weight();
+
+  /**
+   * Collect terms from the most highly-weighted path below this node
+   */
+  public abstract void collectTerms(BiConsumer<String, BytesRef> termCollector);
+
+  /**
+   * Find the next-most highly-weighted path below this node
+   * @param minWeight do not advance if the next path has a weight below this value
+   * @return {@code false} if there are no more paths above the minimum weight
+   */
+  public abstract boolean advancePhase(double minWeight);
+
+  /**
+   * Returns a string representation of the node
+   * @param depth the current depth of this node in the overall query tree
+   */
+  public abstract String toString(int depth);
+
+  @Override
+  public String toString() {
+    return toString(0);
+  }
+
+  /**
+   * Returns a string of {@code width} spaces
+   */
+  protected String space(int width) {
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < width; i++) {
+      sb.append(" ");
+    }
+    return sb.toString();
+  }
+
+  /**
+   * Returns a leaf node for a particular term
+   */
+  public static QueryTree term(Term term, TermWeightor weightor) {
+    return term(term.field(), term.bytes(), weightor.applyAsDouble(term));
+  }
+
+  /**
+   * Returns a leaf node for a particular term and weight
+   *
+   * The weight must be greater than 0
+   */
+  public static QueryTree term(Term term, double weight) {
+    return term(term.field(), term.bytes(), weight);
+  }
+
+  /**
+   * Returns a leaf node for a particular term and weight
+   *
+   * The weight must be greater than 0
+   */
+  public static QueryTree term(String field, BytesRef term, double weight) {
+    return new QueryTree() {
+      @Override
+      public double weight() {
+        if (weight <= 0) {
+          throw new IllegalArgumentException("Term weights must be greater than 0");
+        }
+        return weight;
+      }
+
+      @Override
+      public void collectTerms(BiConsumer<String, BytesRef> termCollector) {
+        termCollector.accept(field, term);
+      }
+
+      @Override
+      public boolean advancePhase(double minWeight) {
+        return false;
+      }
+
+      @Override
+      public String toString(int depth) {
+        return space(depth) + field + ":" + term.utf8ToString() + "^" + weight;
+      }
+    };
+  }
+
+  /**
+   * Returns a leaf node that will match any document
+   */
+  public static QueryTree anyTerm(String reason) {
+    return new QueryTree() {
+      @Override
+      public double weight() {
+        return 0;
+      }
+
+      @Override
+      public void collectTerms(BiConsumer<String, BytesRef> termCollector) {
+        termCollector.accept(TermFilteredPresearcher.ANYTOKEN_FIELD, new BytesRef(TermFilteredPresearcher.ANYTOKEN));
+      }
+
+      @Override
+      public boolean advancePhase(double minWeight) {
+        return false;
+      }
+
+      @Override
+      public String toString(int depth) {
+        return space(depth) + "ANY[" + reason + "]";
+      }
+    };
+  }
+
+  /**
+   * Returns a conjunction of a set of child nodes
+   */
+  public static QueryTree conjunction(List<Function<TermWeightor, QueryTree>> children, TermWeightor weightor) {
+    if (children.size() == 0) {
+      throw new IllegalArgumentException("Cannot build a conjunction with no children");
+    }
+    if (children.size() == 1) {
+      return children.get(0).apply(weightor);
+    }
+    List<QueryTree> qt = children.stream()
+        .map(f -> f.apply(weightor)).collect(Collectors.toList());
+    List<QueryTree> restricted = qt.stream().filter(t -> t.weight() > 0).collect(Collectors.toList());
+    if (restricted.size() == 0) {
+      // all children are ANY, so just return the first one
+      return qt.get(0);
+    }
+    return new ConjunctionQueryTree(qt);
+  }
+
+  static QueryTree conjunction(QueryTree... children) {
+    return new ConjunctionQueryTree(Arrays.asList(children));
+  }
+
+  /**
+   * Returns a disjunction of a set of child nodes
+   */
+  public static QueryTree disjunction(List<Function<TermWeightor, QueryTree>> children, TermWeightor weightor) {
+    if (children.size() == 0) {
+      throw new IllegalArgumentException("Cannot build a disjunction with no children");
+    }
+    if (children.size() == 1) {
+      return children.get(0).apply(weightor);
+    }
+    List<QueryTree> qt = children.stream()
+        .map(f -> f.apply(weightor)).collect(Collectors.toList());
+    Optional<QueryTree> firstAnyChild = qt.stream().filter(q -> q.weight() == 0).findAny();
+    // if any of the children is an ANY node, just return that, otherwise build the disjunction
+    return firstAnyChild.orElseGet(() -> new DisjunctionQueryTree(qt));
+  }
+
+  static QueryTree disjunction(QueryTree... children) {
+    return new DisjunctionQueryTree(Arrays.asList(children));
+  }
+
+  private static class ConjunctionQueryTree extends QueryTree {
+
+    private static final Comparator<QueryTree> COMPARATOR = Comparator.comparingDouble(QueryTree::weight).reversed();
+
+    final List<QueryTree> children = new ArrayList<>();
+
+    ConjunctionQueryTree(List<QueryTree> children) {
+      this.children.addAll(children);
+      this.children.sort(COMPARATOR);
+    }
+
+    @Override
+    public double weight() {
+      return children.get(0).weight();
+    }
+
+    @Override
+    public void collectTerms(BiConsumer<String, BytesRef> termCollector) {
+      children.get(0).collectTerms(termCollector);
+    }
+
+    @Override
+    public boolean advancePhase(double minWeight) {
+      if (children.get(0).advancePhase(minWeight)) {
+        this.children.sort(COMPARATOR);
+        return true;
+      }
+      if (children.size() == 1) {
+        return false;
+      }
+      if (children.get(1).weight() <= minWeight) {
+        return false;
+      }
+      children.remove(0);
+      return true;
+    }
+
+    @Override
+    public String toString(int depth) {
+      StringBuilder sb = new StringBuilder(space(depth)).append("Conjunction[")
+          .append(children.size())
+          .append("]^")
+          .append(weight())
+          .append("\n");
+      for (QueryTree child : children) {
+        sb.append(child.toString(depth + 2)).append("\n");
+      }
+      return sb.toString();
+    }
+  }
+
+  private static class DisjunctionQueryTree extends QueryTree {
+
+    final List<QueryTree> children = new ArrayList<>();
+
+    private DisjunctionQueryTree(List<QueryTree> children) {
+      this.children.addAll(children);
+      this.children.sort(Comparator.comparingDouble(QueryTree::weight));
+    }
+
+    @Override
+    public double weight() {
+      return children.get(0).weight();
+    }
+
+    @Override
+    public void collectTerms(BiConsumer<String, BytesRef> termCollector) {
+      for (QueryTree child : children) {
+        child.collectTerms(termCollector);
+      }
+    }
+
+    @Override
+    public boolean advancePhase(double minWeight) {
+      boolean changed = false;
+      for (QueryTree child : children) {
+        changed |= child.advancePhase(minWeight);
+      }
+      if (changed == false) {
+        return false;
+      }
+      children.sort(Comparator.comparingDouble(QueryTree::weight));
+      return true;
+    }
+
+    @Override
+    public String toString(int depth) {
+      StringBuilder sb = new StringBuilder(space(depth)).append("Disjunction[");
+      sb.append(children.size()).append("]^");
+      sb.append(weight()).append("\n");
+      for (QueryTree child : children) {
+        sb.append(child.toString(depth + 2)).append("\n");
+      }
+      return sb.toString();
+    }
+  }
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/RegexpQueryHandler.java b/lucene/monitor/src/java/org/apache/lucene/monitor/RegexpQueryHandler.java
new file mode 100644
index 0000000..cdc436d
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/RegexpQueryHandler.java
@@ -0,0 +1,155 @@
+/*
+ * 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.util.Collections;
+import java.util.Set;
+import java.util.function.BiConsumer;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.RegexpQuery;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * A query handler implementation that matches Regexp queries by indexing regex
+ * terms by their longest static substring, and generates ngrams from Document
+ * tokens to match them.
+ * <p>
+ * This implementation will filter out more wildcard queries than TermFilteredPresearcher,
+ * at the expense of longer document build times.  Which one is more performant will depend
+ * on the type and number of queries registered in the Monitor, and the size of documents
+ * to be monitored.  Profiling is recommended.
+ */
+public class RegexpQueryHandler implements CustomQueryHandler {
+
+  /**
+   * The default suffix with which to mark ngrams
+   */
+  public static final String DEFAULT_NGRAM_SUFFIX = "XX";
+
+  /**
+   * The default maximum length of an input token before ANYTOKENS are generated
+   */
+  public static final int DEFAULT_MAX_TOKEN_SIZE = 30;
+
+  /**
+   * The default token to emit if a term is longer than MAX_TOKEN_SIZE
+   */
+  public static final String DEFAULT_WILDCARD_TOKEN = "__WILDCARD__";
+
+  private final String ngramSuffix;
+
+  private final String wildcardToken;
+  private final BytesRef wildcardTokenBytes;
+
+  private final int maxTokenSize;
+
+  private final Set<String> excludedFields;
+
+  /**
+   * Creates a new RegexpQueryHandler
+   *
+   * @param ngramSuffix    the suffix with which to mark ngrams
+   * @param maxTokenSize   the maximum length of an input token before WILDCARD tokens are generated
+   * @param wildcardToken  the token to emit if a token is longer than maxTokenSize in length
+   * @param excludedFields a Set of fields to ignore when generating ngrams
+   */
+  public RegexpQueryHandler(String ngramSuffix, int maxTokenSize, String wildcardToken, Set<String> excludedFields) {
+    this.ngramSuffix = ngramSuffix;
+    this.maxTokenSize = maxTokenSize;
+    this.wildcardTokenBytes = new BytesRef(wildcardToken);
+    this.wildcardToken = wildcardToken;
+    this.excludedFields = excludedFields == null ? Collections.emptySet() : excludedFields;
+  }
+
+  /**
+   * Creates a new RegexpQueryHandler using default settings
+   */
+  public RegexpQueryHandler() {
+    this(DEFAULT_NGRAM_SUFFIX, DEFAULT_MAX_TOKEN_SIZE, DEFAULT_WILDCARD_TOKEN, null);
+  }
+
+  /**
+   * Creates a new RegexpQueryHandler with a maximum token size
+   *
+   * @param maxTokenSize the maximum length of an input token before WILDCARD tokens are generated
+   */
+  public RegexpQueryHandler(int maxTokenSize) {
+    this(DEFAULT_NGRAM_SUFFIX, maxTokenSize, DEFAULT_WILDCARD_TOKEN, null);
+  }
+
+  @Override
+  public TokenStream wrapTermStream(String field, TokenStream ts) {
+    if (excludedFields.contains(field))
+      return ts;
+    return new SuffixingNGramTokenFilter(ts, ngramSuffix, wildcardToken, maxTokenSize);
+  }
+
+  @Override
+  public QueryTree handleQuery(Query q, TermWeightor termWeightor) {
+    if (q instanceof RegexpQuery == false) {
+      return null;
+    }
+    RegexpQuery query = (RegexpQuery) q;
+    String regexp = parseOutRegexp(query.toString(""));
+    String selected = selectLongestSubstring(regexp);
+    Term term = new Term(query.getField(), selected + ngramSuffix);
+    double weight = termWeightor.applyAsDouble(term);
+    return new QueryTree() {
+      @Override
+      public double weight() {
+        return weight;
+      }
+
+      @Override
+      public void collectTerms(BiConsumer<String, BytesRef> termCollector) {
+        termCollector.accept(term.field(), term.bytes());
+        termCollector.accept(term.field(), wildcardTokenBytes);
+      }
+
+      @Override
+      public boolean advancePhase(double minWeight) {
+        return false;
+      }
+
+      @Override
+      public String toString(int depth) {
+        return space(depth) + "WILDCARD_NGRAM[" + term.toString() + "]^" + weight;
+      }
+    };
+  }
+
+  private static String parseOutRegexp(String rep) {
+    int fieldSepPos = rep.indexOf(":");
+    int firstSlash = rep.indexOf("/", fieldSepPos);
+    int lastSlash = rep.lastIndexOf("/");
+    return rep.substring(firstSlash + 1, lastSlash);
+  }
+
+  private static String selectLongestSubstring(String regexp) {
+    String selected = "";
+    for (String substr : regexp.split("\\.|\\*|.\\?")) {
+      if (substr.length() > selected.length()) {
+        selected = substr;
+      }
+    }
+    return selected;
+  }
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/ScoringMatch.java b/lucene/monitor/src/java/org/apache/lucene/monitor/ScoringMatch.java
new file mode 100644
index 0000000..80684fd
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/ScoringMatch.java
@@ -0,0 +1,81 @@
+/*
+ * 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.search.Scorable;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.similarities.BM25Similarity;
+import org.apache.lucene.search.similarities.Similarity;
+
+/**
+ * A QueryMatch that reports scores for each match
+ */
+public class ScoringMatch extends QueryMatch {
+
+  public static final MatcherFactory<ScoringMatch> matchWithSimilarity(Similarity similarity) {
+    return searcher -> {
+      searcher.setSimilarity(similarity);
+      return new CollectingMatcher<ScoringMatch>(searcher, ScoreMode.COMPLETE) {
+        @Override
+        protected ScoringMatch doMatch(String queryId, int doc, Scorable scorer) throws IOException {
+          float score = scorer.score();
+          if (score > 0)
+            return new ScoringMatch(queryId, score);
+          return null;
+        }
+
+        @Override
+        public ScoringMatch resolve(ScoringMatch match1, ScoringMatch match2) {
+          return new ScoringMatch(match1.getQueryId(), match1.getScore() + match2.getScore());
+        }
+      };
+    };
+  }
+
+  public static final MatcherFactory<ScoringMatch> DEFAULT_MATCHER = matchWithSimilarity(new BM25Similarity());
+
+  private final float score;
+
+  private ScoringMatch(String queryId, float score) {
+    super(queryId);
+    this.score = score;
+  }
+
+  public float getScore() {
+    return score;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (!(o instanceof ScoringMatch)) return false;
+    if (!super.equals(o)) return false;
+    ScoringMatch that = (ScoringMatch) o;
+    return Float.compare(that.score, score) == 0;
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    result = 31 * result + (score != +0.0f ? Float.floatToIntBits(score) : 0);
+    return result;
+  }
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/SlowLog.java b/lucene/monitor/src/java/org/apache/lucene/monitor/SlowLog.java
new file mode 100644
index 0000000..c8a8618
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/SlowLog.java
@@ -0,0 +1,88 @@
+/*
+ * 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.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+/**
+ * Reports on slow queries in a given match run
+ */
+public class SlowLog implements Iterable<SlowLog.Entry> {
+
+  private final List<Entry> slowQueries = new ArrayList<>();
+
+  /**
+   * Add a query and time taken to the slow log.
+   * <p>
+   * The query will only be recorded if the time is above the configured limit
+   *
+   * @param query the query id
+   * @param time  the time taken by the query in ns
+   */
+  void addQuery(String query, long time) {
+    slowQueries.add(new Entry(query, time));
+  }
+
+  /**
+   * Add all entries to this slow log
+   *
+   * @param queries the entries to add
+   */
+  void addAll(Iterable<SlowLog.Entry> queries) {
+    for (SlowLog.Entry query : queries) {
+      slowQueries.add(query);
+    }
+  }
+
+  @Override
+  public Iterator<Entry> iterator() {
+    return slowQueries.iterator();
+  }
+
+  /**
+   * An individual entry in the slow log
+   */
+  public static class Entry {
+
+    /**
+     * The query id
+     */
+    final String queryId;
+
+    /**
+     * The time taken to execute the query in ms
+     */
+    final long time;
+
+    Entry(String queryId, long time) {
+      this.queryId = queryId;
+      this.time = time;
+    }
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    for (Entry entry : slowQueries) {
+      sb.append(entry.queryId).append(" [").append(entry.time).append("ns]\n");
+    }
+    return sb.toString();
+  }
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/SuffixingNGramTokenFilter.java b/lucene/monitor/src/java/org/apache/lucene/monitor/SuffixingNGramTokenFilter.java
new file mode 100644
index 0000000..f7068b9
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/SuffixingNGramTokenFilter.java
@@ -0,0 +1,144 @@
+/*
+ * 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.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.*;
+import org.apache.lucene.analysis.CharArraySet;
+
+final class SuffixingNGramTokenFilter extends TokenFilter {
+
+  private final String suffix;
+  private final int maxTokenLength;
+  private final String anyToken;
+
+  private char[] curTermBuffer;
+  private int curTermLength;
+  private int curCodePointCount;
+  private int curGramSize;
+  private int curPos;
+  private int curPosInc, curPosLen;
+  private int tokStart;
+  private int tokEnd;
+
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+  private final PositionIncrementAttribute posIncAtt;
+  private final PositionLengthAttribute posLenAtt;
+  private final OffsetAttribute offsetAtt = addAttribute(OffsetAttribute.class);
+  private final KeywordAttribute keywordAtt = addAttribute(KeywordAttribute.class);
+
+  private final CharArraySet seenSuffixes = new CharArraySet(1024, false);
+  private final CharArraySet seenInfixes = new CharArraySet(1024, false);
+
+  /**
+   * Creates SuffixingNGramTokenFilter.
+   *
+   * @param input          {@link org.apache.lucene.analysis.TokenStream} holding the input to be tokenized
+   * @param suffix         a string to suffix to all ngrams
+   * @param wildcardToken  a token to emit if the input token is longer than maxTokenLength
+   * @param maxTokenLength tokens longer than this will not be ngrammed
+   */
+  public SuffixingNGramTokenFilter(TokenStream input, String suffix, String wildcardToken, int maxTokenLength) {
+    super(input);
+
+    this.suffix = suffix;
+    this.anyToken = wildcardToken;
+    this.maxTokenLength = maxTokenLength;
+
+    posIncAtt = addAttribute(PositionIncrementAttribute.class);
+    posLenAtt = addAttribute(PositionLengthAttribute.class);
+
+  }
+
+  /**
+   * Returns the next token in the stream, or null at EOS.
+   */
+  @Override
+  public final boolean incrementToken() throws IOException {
+    while (true) {
+      if (curTermBuffer == null) {
+
+        if (!input.incrementToken()) {
+          return false;
+        }
+
+        if (keywordAtt.isKeyword())
+          return true;
+
+        curTermBuffer = termAtt.buffer().clone();
+        curTermLength = termAtt.length();
+        curCodePointCount = Character.codePointCount(termAtt, 0, termAtt.length());
+        curGramSize = curTermLength;
+        curPos = 0;
+        curPosInc = posIncAtt.getPositionIncrement();
+        curPosLen = posLenAtt.getPositionLength();
+        tokStart = offsetAtt.startOffset();
+        tokEnd = offsetAtt.endOffset();
+        //termAtt.setEmpty().append(suffix);
+        return true;
+
+      }
+
+      if (curTermLength > maxTokenLength) {
+        clearAttributes();
+        termAtt.append(anyToken);
+        curTermBuffer = null;
+        return true;
+      }
+
+      if (curGramSize == 0) {
+        ++curPos;
+        curGramSize = curTermLength - curPos;
+      }
+      if (curGramSize >= 0 && (curPos + curGramSize) <= curCodePointCount) {
+        clearAttributes();
+        final int start = Character.offsetByCodePoints(curTermBuffer, 0, curTermLength, 0, curPos);
+        final int end = Character.offsetByCodePoints(curTermBuffer, 0, curTermLength, start, curGramSize);
+        termAtt.copyBuffer(curTermBuffer, start, end - start);
+        termAtt.append(suffix);
+        if ((curGramSize == curTermLength - curPos) && !seenSuffixes.add(termAtt.subSequence(0, termAtt.length()))) {
+          curTermBuffer = null;
+          continue;
+        }
+        if (!seenInfixes.add(termAtt.subSequence(0, termAtt.length()))) {
+          curGramSize = 0;
+          continue;
+        }
+        posIncAtt.setPositionIncrement(curPosInc);
+        curPosInc = 0;
+        posLenAtt.setPositionLength(curPosLen);
+        offsetAtt.setOffset(tokStart, tokEnd);
+        curGramSize--;
+        return true;
+      }
+
+      curTermBuffer = null;
+    }
+  }
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    curTermBuffer = null;
+    seenInfixes.clear();
+    seenSuffixes.clear();
+  }
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/TermFilteredPresearcher.java b/lucene/monitor/src/java/org/apache/lucene/monitor/TermFilteredPresearcher.java
new file mode 100644
index 0000000..a526f7e
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/TermFilteredPresearcher.java
@@ -0,0 +1,309 @@
+/*
+ * 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.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.BiPredicate;
+
+import org.apache.lucene.analysis.FilteringTokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermInSetQuery;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefHash;
+import org.apache.lucene.util.BytesRefIterator;
+
+/**
+ * Presearcher implementation that uses terms extracted from queries to index
+ * them in the Monitor, and builds a disjunction from terms in a document to match
+ * them.
+ *
+ * Handling of queries that do not support term extraction through the
+ * {@link org.apache.lucene.search.QueryVisitor} API can be configured by passing
+ * a list of {@link CustomQueryHandler} implementations.
+ *
+ * Filtering by additional fields can be configured by passing a set of field names.
+ * Documents that contain values in those fields will only be checked against
+ * {@link MonitorQuery} instances that have the same fieldname-value mapping in
+ * their metadata.
+ */
+public class TermFilteredPresearcher extends Presearcher {
+
+  /**
+   * The default TermWeightor, weighting by token length
+   */
+  public static final TermWeightor DEFAULT_WEIGHTOR = TermWeightor.DEFAULT;
+
+  private final QueryAnalyzer extractor;
+  private final TermWeightor weightor;
+
+  private final Set<String> filterFields;
+  private final List<CustomQueryHandler> queryHandlers = new ArrayList<>();
+
+  static final String ANYTOKEN_FIELD = "__anytokenfield";
+  static final String ANYTOKEN = "__ANYTOKEN__";
+
+  /**
+   * Creates a new TermFilteredPresearcher using the default term weighting
+   */
+  public TermFilteredPresearcher() {
+    this(DEFAULT_WEIGHTOR, Collections.emptyList(), Collections.emptySet());
+  }
+
+  /**
+   * Creates a new TermFilteredPresearcher
+   *
+   * @param weightor            the TermWeightor
+   * @param customQueryHandlers A list of custom query handlers to extract terms from non-core queries
+   * @param filterFields        A set of fields to filter on
+   */
+  public TermFilteredPresearcher(TermWeightor weightor, List<CustomQueryHandler> customQueryHandlers, Set<String> filterFields) {
+    this.extractor = new QueryAnalyzer(customQueryHandlers);
+    this.filterFields = filterFields;
+    this.queryHandlers.addAll(customQueryHandlers);
+    this.weightor = weightor;
+  }
+
+  @Override
+  public final Query buildQuery(LeafReader reader, BiPredicate<String, BytesRef> termAcceptor) {
+    try {
+      DocumentQueryBuilder queryBuilder = getQueryBuilder();
+      for (FieldInfo field : reader.getFieldInfos()) {
+
+        TokenStream ts = new TermsEnumTokenStream(reader.terms(field.name).iterator());
+        for (CustomQueryHandler handler : queryHandlers) {
+          ts = handler.wrapTermStream(field.name, ts);
+        }
+
+        ts = new FilteringTokenFilter(ts) {
+          TermToBytesRefAttribute termAtt = addAttribute(TermToBytesRefAttribute.class);
+          @Override
+          protected boolean accept() {
+            return filterFields.contains(field.name) == false && termAcceptor.test(field.name, termAtt.getBytesRef());
+          }
+        };
+
+        TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
+        while (ts.incrementToken()) {
+          queryBuilder.addTerm(field.name, BytesRef.deepCopyOf(termAtt.getBytesRef()));
+        }
+        ts.close();
+
+      }
+      Query presearcherQuery = queryBuilder.build();
+
+      BooleanQuery.Builder bq = new BooleanQuery.Builder();
+      bq.add(presearcherQuery, BooleanClause.Occur.SHOULD);
+      bq.add(new TermQuery(new Term(ANYTOKEN_FIELD, ANYTOKEN)), BooleanClause.Occur.SHOULD);
+      presearcherQuery = bq.build();
+      if (filterFields.isEmpty() == false) {
+        bq = new BooleanQuery.Builder();
+        bq.add(presearcherQuery, BooleanClause.Occur.MUST);
+        Query filterQuery = buildFilterFields(reader);
+        if (filterQuery != null) {
+          bq.add(filterQuery, BooleanClause.Occur.FILTER);
+          presearcherQuery = bq.build();
+        }
+      }
+      return presearcherQuery;
+    } catch (IOException e) {
+      // We're a MemoryIndex, so this shouldn't happen...
+      throw new RuntimeException(e);
+    }
+  }
+
+  private Query buildFilterFields(LeafReader reader) throws IOException {
+    BooleanQuery.Builder builder = new BooleanQuery.Builder();
+    for (String field : filterFields) {
+      Query q = buildFilterClause(reader, field);
+      if (q != null) {
+        builder.add(q, BooleanClause.Occur.MUST);
+      }
+    }
+    BooleanQuery bq = builder.build();
+    if (bq.clauses().size() == 0) {
+      return null;
+    }
+    return bq;
+  }
+
+  private Query buildFilterClause(LeafReader reader, String field) throws IOException {
+
+    Terms terms = reader.terms(field);
+    if (terms == null)
+      return null;
+
+    BooleanQuery.Builder bq = new BooleanQuery.Builder();
+
+    int docsInBatch = reader.maxDoc();
+
+    BytesRef term;
+    TermsEnum te = terms.iterator();
+    while ((term = te.next()) != null) {
+      // we need to check that every document in the batch has the same field values, otherwise
+      // this filtering will not work
+      if (te.docFreq() != docsInBatch)
+        throw new IllegalArgumentException("Some documents in this batch do not have a term value of "
+            + field + ":" + Term.toString(term));
+      bq.add(new TermQuery(new Term(field, BytesRef.deepCopyOf(term))), BooleanClause.Occur.SHOULD);
+    }
+
+    BooleanQuery built = bq.build();
+
+    if (built.clauses().size() == 0)
+      return null;
+
+    return built;
+  }
+
+  /**
+   * Constructs a document disjunction from a set of terms
+   */
+  protected interface DocumentQueryBuilder {
+
+    /**
+     * Add a term from this document
+     */
+    void addTerm(String field, BytesRef term) throws IOException;
+
+    /**
+     * @return the final Query
+     */
+    Query build();
+
+  }
+
+  /**
+   * Returns a {@link DocumentQueryBuilder} for this presearcher
+   */
+  protected DocumentQueryBuilder getQueryBuilder() {
+    return new DocumentQueryBuilder() {
+
+      Map<String, List<BytesRef>> terms = new HashMap<>();
+
+      @Override
+      public void addTerm(String field, BytesRef term) {
+        List<BytesRef> t = terms.computeIfAbsent(field, f -> new ArrayList<>());
+        t.add(term);
+      }
+
+      @Override
+      public Query build() {
+        BooleanQuery.Builder builder = new BooleanQuery.Builder();
+        for (String field : terms.keySet()) {
+          builder.add(new TermInSetQuery(field, terms.get(field)), BooleanClause.Occur.SHOULD);
+        }
+        return builder.build();
+      }
+    };
+  }
+
+  static final FieldType QUERYFIELDTYPE;
+
+  static {
+    QUERYFIELDTYPE = new FieldType(TextField.TYPE_NOT_STORED);
+    QUERYFIELDTYPE.setIndexOptions(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS);
+    QUERYFIELDTYPE.freeze();
+  }
+
+  @Override
+  public final Document indexQuery(Query query, Map<String, String> metadata) {
+    QueryTree querytree = extractor.buildTree(query, weightor);
+    Document doc = buildQueryDocument(querytree);
+    for (String field : filterFields) {
+      if (metadata != null && metadata.containsKey(field)) {
+        doc.add(new TextField(field, metadata.get(field), Field.Store.YES));
+      }
+    }
+    return doc;
+  }
+
+  /**
+   * Builds a {@link Document} from the terms extracted from a query
+   */
+  protected Document buildQueryDocument(QueryTree querytree) {
+    Map<String, BytesRefHash> fieldTerms = collectTerms(querytree);
+    Document doc = new Document();
+    for (Map.Entry<String, BytesRefHash> entry : fieldTerms.entrySet()) {
+      doc.add(new Field(entry.getKey(),
+          new TermsEnumTokenStream(new BytesRefHashIterator(entry.getValue())), QUERYFIELDTYPE));
+    }
+    return doc;
+  }
+
+  /**
+   * Collects terms from a {@link QueryTree} and maps them per-field
+   */
+  protected Map<String, BytesRefHash> collectTerms(QueryTree querytree) {
+    Map<String, BytesRefHash> fieldTerms = new HashMap<>();
+    querytree.collectTerms((field, term) -> {
+      BytesRefHash tt = fieldTerms.computeIfAbsent(field, f -> new BytesRefHash());
+      tt.add(term);
+    });
+    return fieldTerms;
+  }
+
+  /**
+   * Implements a {@link BytesRefIterator} over a {@link BytesRefHash}
+   */
+  protected class BytesRefHashIterator implements BytesRefIterator {
+
+    final BytesRef scratch = new BytesRef();
+    final BytesRefHash terms;
+    final int[] sortedTerms;
+    int upto = -1;
+
+
+    BytesRefHashIterator(BytesRefHash terms) {
+      this.terms = terms;
+      this.sortedTerms = terms.sort();
+    }
+
+    @Override
+    public BytesRef next() {
+      if (upto >= sortedTerms.length)
+        return null;
+      upto++;
+      if (sortedTerms[upto] == -1)
+        return null;
+      this.terms.get(sortedTerms[upto], scratch);
+      return scratch;
+    }
+  }
+
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/TermWeightor.java b/lucene/monitor/src/java/org/apache/lucene/monitor/TermWeightor.java
new file mode 100644
index 0000000..54c995b
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/TermWeightor.java
@@ -0,0 +1,152 @@
+/*
+ * 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.util.Arrays;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.function.ToDoubleFunction;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Calculates the weight of a {@link Term}
+ */
+public interface TermWeightor extends ToDoubleFunction<Term> {
+
+  /**
+   * A default TermWeightor based on token length
+   */
+  TermWeightor DEFAULT = lengthWeightor(3, 0.3f);
+
+  /**
+   * Combine weightors by multiplication
+   */
+  static TermWeightor combine(TermWeightor... weightors) {
+    return value -> {
+      double r = 1;
+      for (TermWeightor w : weightors) {
+        r *= w.applyAsDouble(value);
+      }
+      return r;
+    };
+  }
+
+  /**
+   * QueryTerms with a field from the selected set will be assigned the given weight
+   */
+  static TermWeightor fieldWeightor(double weight, Set<String> fields) {
+    return value -> {
+      if (fields.contains(value.field())) {
+        return weight;
+      }
+      return 1;
+    };
+  }
+
+  /**
+   * QueryTerms with a field from the selected set will be assigned the given weight
+   */
+  static TermWeightor fieldWeightor(double weight, String... fields) {
+    return fieldWeightor(weight, new HashSet<>(Arrays.asList(fields)));
+  }
+
+  /**
+   * QueryTerms with a term value from the selected set will be assigned the given weight
+   */
+  static TermWeightor termWeightor(double weight, Set<BytesRef> terms) {
+    return value -> {
+      if (terms.contains(value.bytes())) {
+        return weight;
+      }
+      return 1;
+    };
+  }
+
+  /**
+   * QueryTerms with a term value from the selected set will be assigned the given weight
+   */
+  static TermWeightor termWeightor(double weight, BytesRef... terms) {
+    return termWeightor(weight, new HashSet<>(Arrays.asList(terms)));
+  }
+
+  /**
+   * QueryTerms with a term and field value from the selected set will be assigned the given weight
+   */
+  static TermWeightor termAndFieldWeightor(double weight, Set<Term> terms) {
+    return value -> {
+      if (terms.contains(value)) {
+        return weight;
+      }
+      return 1;
+    };
+  }
+
+  /**
+   * QueryTerms with a term and field value from the selected set will be assigned the given weight
+   */
+  static TermWeightor termAndFieldWeightor(double weight, Term... terms) {
+    return termAndFieldWeightor(weight, new HashSet<>(Arrays.asList(terms)));
+  }
+
+  /**
+   * QueryTerms will be assigned a weight based on their term frequency
+   *
+   * More infrequent terms are weighted higher.  Terms are weighted according
+   * to the function {@code w = (n / freq) + k}.  Terms with no associated
+   * frequency receive a weight of value {@code 1}
+   *
+   * @param frequencies a map of terms to frequencies
+   * @param n           a scaling factor
+   * @param k           the minimum weight to scale to
+   */
+  static TermWeightor termFreqWeightor(Map<String, Integer> frequencies, double n, double k) {
+    return value -> {
+      Integer mapVal = frequencies.get(value.text());
+      if (mapVal != null)
+        return (n / mapVal) + k;
+      return 1;
+    };
+  }
+
+  /**
+   * QueryTerms will be assigned a weight based on their term length
+   *
+   * Weights are assigned by the function {@code a * e ^ (-k * length)}. Longer
+   * terms are weighted higher. Terms of length greater than 32 all receive the
+   * same weight.
+   *
+   * @param a a
+   * @param k k
+   */
+  static TermWeightor lengthWeightor(double a, double k) {
+    final double[] lengthNorms = new double[32];
+    for (int i = 0; i < 32; i++) {
+      lengthNorms[i] = (float) (a * (Math.exp(-k * i)));
+    }
+    return value -> {
+      if (value.bytes().length >= 32) {
+        return 4 - lengthNorms[31];
+      }
+      return 4 - lengthNorms[value.bytes().length];
+    };
+  }
+
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/TermsEnumTokenStream.java b/lucene/monitor/src/java/org/apache/lucene/monitor/TermsEnumTokenStream.java
new file mode 100644
index 0000000..66c66d1
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/TermsEnumTokenStream.java
@@ -0,0 +1,54 @@
+/*
+ * 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.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefIterator;
+
+/**
+ * A TokenStream created from a {@link org.apache.lucene.index.TermsEnum}
+ */
+class TermsEnumTokenStream extends TokenStream {
+
+  private final BytesRefIterator termsEnum;
+  private final CharTermAttribute charTerm = addAttribute(CharTermAttribute.class);
+
+  /**
+   * Create a new TermsEnumTokenStream using a TermsEnum
+   *
+   * @param termsEnum the TermsEnum to convert
+   */
+  public TermsEnumTokenStream(BytesRefIterator termsEnum) {
+    this.termsEnum = termsEnum;
+  }
+
+  @Override
+  public final boolean incrementToken() throws IOException {
+    clearAttributes();
+    BytesRef bytes = termsEnum.next();
+    if (bytes == null)
+      return false;
+    charTerm.setEmpty();
+    charTerm.append(bytes.utf8ToString());
+    return true;
+  }
+}
diff --git a/lucene/monitor/src/java/org/apache/lucene/monitor/package-info.java b/lucene/monitor/src/java/org/apache/lucene/monitor/package-info.java
new file mode 100644
index 0000000..8f6d7c8
--- /dev/null
+++ b/lucene/monitor/src/java/org/apache/lucene/monitor/package-info.java
@@ -0,0 +1,98 @@
+/*
+ * 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.
+ */
+
+/**
+ * <h2>Monitoring framework</h2>
+ *
+ * This package contains classes to allow the monitoring of a stream of
+ * documents with a set of queries.
+ *
+ * To use, instantiate a {@link org.apache.lucene.monitor.Monitor} object,
+ * register queries with it via
+ * {@link org.apache.lucene.monitor.Monitor#register(org.apache.lucene.monitor.MonitorQuery...)},
+ * and then match documents against it either invidually via
+ * {@link org.apache.lucene.monitor.Monitor#match(org.apache.lucene.document.Document, org.apache.lucene.monitor.MatcherFactory)}
+ * or in batches via
+ * {@link org.apache.lucene.monitor.Monitor#match(org.apache.lucene.document.Document[], org.apache.lucene.monitor.MatcherFactory)}
+ *
+ * <h3>Matcher types</h3>
+ *
+ * A number of matcher types are included:
+ * <ul>
+ *   <li>{@link org.apache.lucene.monitor.QueryMatch#SIMPLE_MATCHER} &mdash; just returns the set of query ids that a Document has matched</li>
+ *   <li>{@link org.apache.lucene.monitor.ScoringMatch#matchWithSimilarity(org.apache.lucene.search.similarities.Similarity)}
+ *          &mdash; returns the set of matching queries, with the score that each one records against a Document</li>
+ *   <li>{@link org.apache.lucene.monitor.ExplainingMatch#MATCHER &mdash; similar to ScoringMatch, but include the full Explanation}</li>
+ *   <li>{@link org.apache.lucene.monitor.HighlightsMatch#MATCHER &mdash; return the matching queries along with the matching terms for each query}</li>
+ * </ul>
+ *
+ * Matchers can be wrapped in {@link org.apache.lucene.monitor.PartitionMatcher} or {@link org.apache.lucene.monitor.ParallelMatcher} to increase
+ * performance in low-concurrency systems.
+ *
+ * <h3>Pre-filtering of queries</h3>
+ *
+ * Monitoring is done efficiently by extracting minimal sets of terms from queries, and using these
+ * to build a query index.  When a document is passed to
+ * {@link org.apache.lucene.monitor.Monitor#match(org.apache.lucene.document.Document, org.apache.lucene.monitor.MatcherFactory)},
+ * it is converted into a small index, and the terms dictionary from that index is then used to build
+ * a disjunction query to run against the query index.  Queries that match this disjunction are then run
+ * against the document.  In this way, the Monitor can avoid running queries that have no chance of
+ * matching.  The process of extracting terms and building document disjunctions is handled by a
+ * {@link org.apache.lucene.monitor.Presearcher}
+ *
+ * In addition, extra per-field filtering can be specified by passing a set of keyword fields to
+ * filter on.  When queries are registered with the monitor, field-value pairs can be added as
+ * optional metadata for each query, and these can then be used to restrict which queries a
+ * document is checked against.  For example, you can specify a language that each query should
+ * apply to, and documents containing a value in their language field would only be checked against
+ * queries that have that same value in their language metadata.  Note that when matching documents
+ * in batches, all documents in the batch must have the same values in their filter fields.
+ *
+ * Query analysis uses the {@link org.apache.lucene.search.QueryVisitor} API to extract terms, which will work
+ * for all basic term-based queries shipped with Lucene.  The analyzer builds a representation of the query
+ * called a {@link org.apache.lucene.monitor.QueryTree}, and then selects a minimal set of terms, one of which
+ * must be present in a document for that document to match.  Individual terms are weighted using a
+ * {@link org.apache.lucene.monitor.TermWeightor}, which allows some selectivity when building the term set.
+ * For example, given a conjunction of terms (a boolean query with several MUST clauses, or a phrase, span or interval
+ * query), we need only extract one term.  The TermWeightor can be configured in a number of ways; by default
+ * it will weight longer terms more highly.
+ *
+ * For query sets that contain many conjunctions, it can be useful to extract and index different
+ * minimal term combinations.  For example, a phrase query on 'the quick brown fox' could index
+ * both 'quick' and 'brown', and avoid being run againt documents that contain only one of these
+ * terms.  The {@link org.apache.lucene.monitor.MultipassTermFilteredPresearcher} allows this sort
+ * of indexing, taking a minimum term weight so that very common terms such as 'the' can be avoided.
+ *
+ * Custom Query implementations that are based on term matching, and that implement
+ * {@link org.apache.lucene.search.Query#visit(org.apache.lucene.search.QueryVisitor)} will work with no
+ * extra configuration; for more complicated custom queries, you can register a
+ * {@link org.apache.lucene.monitor.CustomQueryHandler} with the presearcher.  Included in this package
+ * is a {@link org.apache.lucene.monitor.RegexpQueryHandler}, which gives an example of a different method
+ * of indexing automaton-based queries by extracting fixed substrings from a regular expression, and then
+ * using ngram filtering to build the document disjunction.
+ *
+ * <h3>Persistent query sets</h3>
+ *
+ * By default, {@link org.apache.lucene.monitor.Monitor} instances are ephemeral, storing their query
+ * indexes in memory.  To make a persistent monitor, build a {@link org.apache.lucene.monitor.MonitorConfiguration}
+ * object and call {@link org.apache.lucene.monitor.MonitorConfiguration#setIndexPath(java.nio.file.Path, org.apache.lucene.monitor.MonitorQuerySerializer)}
+ * to tell the Monitor to store its query index on disk.  All queries registered with this Monitor will
+ * need to have a string representation that is also stored, and can be re-parsed by the associated
+ * {@link org.apache.lucene.monitor.MonitorQuerySerializer} when the index is loaded by a new Monitor
+ * instance.
+ */
+package org.apache.lucene.monitor;
\ No newline at end of file
diff --git a/lucene/monitor/src/java/overview.html b/lucene/monitor/src/java/overview.html
new file mode 100644
index 0000000..c886115
--- /dev/null
+++ b/lucene/monitor/src/java/overview.html
@@ -0,0 +1,26 @@
+<!--
+ 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.
+-->
+<html>
+  <head>
+    <title>
+      monitor
+    </title>
+  </head>
+  <body>
+  Monitoring framework for lucene queries
+  </body>
+</html>
\ No newline at end of file
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/ConcurrentMatcherTestBase.java b/lucene/monitor/src/test/org/apache/lucene/monitor/ConcurrentMatcherTestBase.java
new file mode 100644
index 0000000..7a120d0
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/ConcurrentMatcherTestBase.java
@@ -0,0 +1,96 @@
+/*
+ * 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.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.NamedThreadFactory;
+
+public abstract class ConcurrentMatcherTestBase extends LuceneTestCase {
+
+  private static final Analyzer ANALYZER = new StandardAnalyzer();
+
+  protected abstract <T extends QueryMatch> MatcherFactory<T> matcherFactory(ExecutorService executor,
+                                                                             MatcherFactory<T> factory, int threads);
+
+  public void testAllMatchesAreCollected() throws Exception {
+
+    ExecutorService executor = Executors.newFixedThreadPool(10, new NamedThreadFactory("matchers"));
+    try (Monitor monitor = new Monitor(ANALYZER)) {
+      List<MonitorQuery> queries = new ArrayList<>();
+      for (int i = 0; i < 1000; i++) {
+        queries.add(new MonitorQuery(Integer.toString(i), MonitorTestBase.parse("+test " + i)));
+      }
+      monitor.register(queries);
+
+      Document doc = new Document();
+      doc.add(newTextField("field", "test", Field.Store.NO));
+
+      MatchingQueries<QueryMatch> matches
+          = monitor.match(doc, matcherFactory(executor, QueryMatch.SIMPLE_MATCHER, 10));
+
+      assertEquals(1000, matches.getMatchCount());
+    }
+    finally {
+      executor.shutdown();
+    }
+  }
+
+  public void testMatchesAreDisambiguated() throws Exception {
+
+    ExecutorService executor = Executors.newFixedThreadPool(4, new NamedThreadFactory("matchers"));
+
+    try (Monitor monitor = new Monitor(ANALYZER)) {
+      List<MonitorQuery> queries = new ArrayList<>();
+      for (int i = 0; i < 10; i++) {
+        queries.add(new MonitorQuery(Integer.toString(i), MonitorTestBase.parse("test^10 doc " + i)));
+      }
+      monitor.register(queries);
+      assertEquals(30, monitor.getDisjunctCount());
+
+      Document doc = new Document();
+      doc.add(newTextField("field", "test doc doc", Field.Store.NO));
+
+      MatchingQueries<ScoringMatch> matches
+          = monitor.match(doc, matcherFactory(executor, ScoringMatch.DEFAULT_MATCHER, 10));
+
+      assertEquals(20, matches.getQueriesRun());
+      assertEquals(10, matches.getMatchCount());
+      assertTrue(matches.getErrors().isEmpty());
+      for (ScoringMatch match : matches.getMatches()) {
+        // The queries are all split into three by the QueryDecomposer, and the
+        // 'test' and 'doc' parts will match.  'test' will have a higher score,
+        // because of it's lower termfreq.  We need to check that each query ends
+        // up with the sum of the scores for the 'test' and 'doc' parts
+        assertEquals(1.4874471f, match.getScore(), 0);
+      }
+    }
+    finally {
+      executor.shutdown();
+    }
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/FieldFilterPresearcherComponentTestBase.java b/lucene/monitor/src/test/org/apache/lucene/monitor/FieldFilterPresearcherComponentTestBase.java
new file mode 100644
index 0000000..da059b8
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/FieldFilterPresearcherComponentTestBase.java
@@ -0,0 +1,141 @@
+/*
+ * 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.Collections;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.search.MatchAllDocsQuery;
+
+import static org.hamcrest.CoreMatchers.containsString;
+
+public abstract class FieldFilterPresearcherComponentTestBase extends PresearcherTestBase {
+
+  public void testBatchFiltering() throws IOException {
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(
+          mq("1", "test", "language", "en"),
+          mq("2", "wahl", "language", "de"),
+          mq("3", "wibble", "language", "en"),
+          mq("4", "*:*", "language", "de"),
+          mq("5", "*:*", "language", "es"));
+
+      Document doc1 = new Document();
+      doc1.add(newTextField(TEXTFIELD, "this is a test", Field.Store.NO));
+      doc1.add(newTextField("language", "en", Field.Store.NO));
+      Document doc2 = new Document();
+      doc2.add(newTextField(TEXTFIELD, "this is a wibble", Field.Store.NO));
+      doc2.add(newTextField("language", "en", Field.Store.NO));
+      Document doc3 = new Document();
+      doc3.add(newTextField(TEXTFIELD, "wahl is a misspelling of whale", Field.Store.NO));
+      doc3.add(newTextField("language", "en", Field.Store.NO));
+
+      MultiMatchingQueries<QueryMatch> matches = monitor.match(new Document[]{ doc1, doc2, doc3 }, QueryMatch.SIMPLE_MATCHER);
+      assertEquals(1, matches.getMatchCount(0));
+      assertNotNull(matches.matches("1", 0));
+      assertEquals(1, matches.getMatchCount(1));
+      assertNotNull(matches.matches("3", 1));
+      assertEquals(0, matches.getMatchCount(2));
+      assertEquals(2, matches.getQueriesRun());
+    }
+  }
+
+  public void testBatchesWithDissimilarFieldValuesThrowExceptions() throws IOException {
+
+    Document doc1 = new Document();
+    doc1.add(newTextField(TEXTFIELD, "test", Field.Store.NO));
+    doc1.add(newTextField("language", "en", Field.Store.NO));
+    Document doc2 = new Document();
+    doc2.add(newTextField(TEXTFIELD, "test", Field.Store.NO));
+    doc2.add(newTextField("language", "de", Field.Store.NO));
+
+    try (Monitor monitor = newMonitor()) {
+      IllegalArgumentException e
+          = expectThrows(IllegalArgumentException.class, () -> monitor.match(new Document[]{ doc1, doc2 }, QueryMatch.SIMPLE_MATCHER));
+      assertThat(e.getMessage(), containsString("language:"));
+    }
+  }
+
+  public void testFieldFiltering() throws IOException {
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(
+          new MonitorQuery("1", parse("test"), null, Collections.singletonMap("language", "en")),
+          new MonitorQuery("2", parse("test"), null, Collections.singletonMap("language", "de")),
+          new MonitorQuery("3", parse("wibble"), null, Collections.singletonMap("language", "en")),
+          new MonitorQuery("4", parse("*:*"), null, Collections.singletonMap("language", "de")));
+
+      Document enDoc = new Document();
+      enDoc.add(newTextField(TEXTFIELD, "this is a test", Field.Store.NO));
+      enDoc.add(newTextField("language", "en", Field.Store.NO));
+
+      MatchingQueries<QueryMatch> en = monitor.match(enDoc, QueryMatch.SIMPLE_MATCHER);
+      assertEquals(1, en.getMatchCount());
+      assertNotNull(en.matches("1"));
+      assertEquals(1, en.getQueriesRun());
+
+      Document deDoc = new Document();
+      deDoc.add(newTextField(TEXTFIELD, "das ist ein test", Field.Store.NO));
+      deDoc.add(newTextField("language", "de", Field.Store.NO));
+
+      MatchingQueries<QueryMatch> de = monitor.match(deDoc, QueryMatch.SIMPLE_MATCHER);
+      assertEquals(2, de.getMatchCount());
+      assertEquals(2, de.getQueriesRun());
+      assertNotNull(de.matches("2"));
+      assertNotNull(de.matches("4"));
+
+      Document bothDoc = new Document();
+      bothDoc.add(newTextField(TEXTFIELD, "this is ein test", Field.Store.NO));
+      bothDoc.add(newTextField("language", "en", Field.Store.NO));
+      bothDoc.add(newTextField("language", "de", Field.Store.NO));
+
+      MatchingQueries<QueryMatch> both = monitor.match(bothDoc, QueryMatch.SIMPLE_MATCHER);
+      assertEquals(3, both.getMatchCount());
+      assertEquals(3, both.getQueriesRun());
+    }
+  }
+
+  public void testFilteringOnMatchAllQueries() throws IOException {
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", new MatchAllDocsQuery(), null, Collections.singletonMap("language", "de")));
+
+      Document enDoc = new Document();
+      enDoc.add(newTextField(TEXTFIELD, "this is a test", Field.Store.NO));
+      enDoc.add(newTextField("language", "en", Field.Store.NO));
+      MatchingQueries<QueryMatch> matches = monitor.match(enDoc, QueryMatch.SIMPLE_MATCHER);
+      assertEquals(0, matches.getMatchCount());
+      assertEquals(0, matches.getQueriesRun());
+    }
+  }
+
+  public void testDebugQueries() throws Exception {
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", parse("test"), null, Collections.singletonMap("language", "en")));
+
+      Document enDoc = new Document();
+      enDoc.add(newTextField(TEXTFIELD, "this is a test", Field.Store.NO));
+      enDoc.add(newTextField("language", "en", Field.Store.NO));
+
+      PresearcherMatches<QueryMatch> matches = monitor.debug(enDoc, QueryMatch.SIMPLE_MATCHER);
+      assertFalse(matches.match("1", 0).presearcherMatches.isEmpty());
+    }
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/MonitorTestBase.java b/lucene/monitor/src/test/org/apache/lucene/monitor/MonitorTestBase.java
new file mode 100644
index 0000000..a6e18fc
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/MonitorTestBase.java
@@ -0,0 +1,93 @@
+/*
+ * 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.Map;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.queryparser.classic.ParseException;
+import org.apache.lucene.queryparser.classic.QueryParser;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryVisitor;
+import org.apache.lucene.util.LuceneTestCase;
+
+public abstract class MonitorTestBase extends LuceneTestCase {
+
+  public static final String FIELD = "field";
+  public static final Analyzer ANALYZER = new StandardAnalyzer();
+
+  public static Query parse(String query) {
+    QueryParser parser = new QueryParser(FIELD, ANALYZER);
+    try {
+      return parser.parse(query);
+    } catch (ParseException e) {
+      throw new IllegalArgumentException(e);
+    }
+  }
+
+  public static MonitorQuery mq(String id, String query, String... metadata) {
+    Query q = parse(query);
+    assert metadata.length % 2 == 0;
+    Map<String, String> mm = new HashMap<>();
+    for (int i = 0; i < metadata.length / 2; i += 2) {
+      mm.put(metadata[i], metadata[i + 1]);
+    }
+    return new MonitorQuery(id, q, query, mm);
+  }
+
+  protected Monitor newMonitor() throws IOException {
+    return newMonitor(new StandardAnalyzer());
+  }
+
+  protected Monitor newMonitor(Analyzer analyzer) throws IOException {
+    // TODO: randomize presearcher
+    return new Monitor(analyzer);
+  }
+
+  public static class ThrowOnRewriteQuery extends Query {
+
+    @Override
+    public Query rewrite(IndexReader reader) throws IOException {
+      throw new IOException("Error rewriting");
+    }
+
+    @Override
+    public String toString(String field) {
+      return "ThrowOnRewriteQuery";
+    }
+
+    @Override
+    public void visit(QueryVisitor visitor) {
+      visitor.visitLeaf(this);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      return false;
+    }
+
+    @Override
+    public int hashCode() {
+      return 0;
+    }
+  }
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/PresearcherTestBase.java b/lucene/monitor/src/test/org/apache/lucene/monitor/PresearcherTestBase.java
new file mode 100644
index 0000000..5fff667
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/PresearcherTestBase.java
@@ -0,0 +1,178 @@
+/*
+ * 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.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.util.AttributeImpl;
+import org.apache.lucene.util.AttributeReflector;
+import org.apache.lucene.util.BytesRef;
+
+public abstract class PresearcherTestBase extends MonitorTestBase {
+
+  public Monitor newMonitor() throws IOException {
+    return new Monitor(WHITESPACE, createPresearcher());
+  }
+
+  protected abstract Presearcher createPresearcher();
+
+  static final String TEXTFIELD = FIELD;
+  static final Analyzer WHITESPACE = new WhitespaceAnalyzer();
+
+  public static Document buildDoc(String field, String text) {
+    Document doc = new Document();
+    doc.add(newTextField(field, text, Field.Store.NO));
+    return doc;
+  }
+
+  public void testNullFieldHandling() throws IOException {
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", parse("field_1:test")));
+
+      assertEquals(0,
+          monitor.match(buildDoc("field_2", "test"), QueryMatch.SIMPLE_MATCHER).getMatchCount());
+    }
+
+  }
+
+  public void testEmptyMonitorHandling() throws IOException {
+    try (Monitor monitor = newMonitor()) {
+      MatchingQueries<QueryMatch> matches = monitor.match(buildDoc("field_2", "test"), QueryMatch.SIMPLE_MATCHER);
+      assertEquals(0, matches.getMatchCount());
+      assertEquals(0, matches.getQueriesRun());
+    }
+  }
+
+  public void testMatchAllQueryHandling() throws IOException {
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", new MatchAllDocsQuery()));
+      assertEquals(1,
+          monitor.match(buildDoc("f", "wibble"), QueryMatch.SIMPLE_MATCHER).getMatchCount());
+    }
+  }
+
+  public void testNegativeQueryHandling() throws IOException {
+    Query q = new BooleanQuery.Builder()
+        .add(new MatchAllDocsQuery(), BooleanClause.Occur.SHOULD)
+        .add(new TermQuery(new Term("f", "foo")), BooleanClause.Occur.MUST_NOT)
+        .build();
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", q));
+
+      MultiMatchingQueries<QueryMatch> matches = monitor.match(new Document[]{
+          buildDoc("f", "bar"), buildDoc("f", "foo")
+      }, QueryMatch.SIMPLE_MATCHER);
+      assertEquals(1, matches.getMatchCount(0));
+      assertEquals(0, matches.getMatchCount(1));
+    }
+  }
+
+  public void testAnyTokenHandling() throws IOException {
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", new MatchAllDocsQuery()));
+      MatchingQueries<QueryMatch> matches = monitor.match(buildDoc("f", "wibble"), QueryMatch.SIMPLE_MATCHER);
+      assertEquals(1, matches.getMatchCount());
+      assertEquals(1, matches.getQueriesRun());
+    }
+  }
+
+  private static final BytesRef NON_STRING_TERM = new BytesRef(new byte[]{60, 8, 0, 0, 0, 9});
+
+  static class BytesRefAttribute extends AttributeImpl implements TermToBytesRefAttribute {
+
+    @Override
+    public BytesRef getBytesRef() {
+      return NON_STRING_TERM;
+    }
+
+    @Override
+    public void clear() {
+
+    }
+
+    @Override
+    public void reflectWith(AttributeReflector attributeReflector) {
+
+    }
+
+    @Override
+    public void copyTo(AttributeImpl attribute) {
+
+    }
+  }
+
+  static final class NonStringTokenStream extends TokenStream {
+
+    final TermToBytesRefAttribute att;
+    boolean done = false;
+
+    NonStringTokenStream() {
+      addAttributeImpl(new BytesRefAttribute());
+      this.att = addAttribute(TermToBytesRefAttribute.class);
+    }
+
+    @Override
+    public boolean incrementToken() {
+      if (done)
+        return false;
+      return done = true;
+    }
+  }
+
+  public void testNonStringTermHandling() throws IOException {
+
+    FieldType ft = new FieldType();
+    ft.setTokenized(true);
+    ft.setIndexOptions(IndexOptions.DOCS_AND_FREQS);
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", new TermQuery(new Term("f", NON_STRING_TERM))));
+
+      Document doc = new Document();
+      doc.add(new Field("f", new NonStringTokenStream(), ft));
+      MatchingQueries<QueryMatch> m = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertEquals(1, m.getMatchCount());
+      assertEquals(1, m.getQueriesRun());
+    }
+
+  }
+
+  public static BooleanClause must(Query q) {
+    return new BooleanClause(q, BooleanClause.Occur.MUST);
+  }
+
+  public static BooleanClause should(Query q) {
+    return new BooleanClause(q, BooleanClause.Occur.SHOULD);
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestBooleanClauseWeightings.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestBooleanClauseWeightings.java
new file mode 100644
index 0000000..87d1a7a
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestBooleanClauseWeightings.java
@@ -0,0 +1,64 @@
+/*
+ * 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.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestBooleanClauseWeightings extends LuceneTestCase {
+
+  private static QueryAnalyzer treeBuilder = new QueryAnalyzer();
+
+  public void testExactClausesPreferred() {
+    Query bq = new BooleanQuery.Builder()
+        .add(LongPoint.newRangeQuery("field2", 1, 2), BooleanClause.Occur.MUST)
+        .add(new BooleanQuery.Builder()
+            .add(new TermQuery(new Term("field1", "term1")), BooleanClause.Occur.SHOULD)
+            .add(new TermQuery(new Term("field1", "term2")), BooleanClause.Occur.SHOULD)
+            .build(), BooleanClause.Occur.MUST)
+        .build();
+    QueryTree tree = treeBuilder.buildTree(bq, TermWeightor.DEFAULT);
+    Set<Term> terms = new HashSet<>();
+    tree.collectTerms((f, b) -> terms.add(new Term(f, b)));
+    assertEquals(2, terms.size());
+  }
+
+  public void testLongerTermsPreferred() {
+    Query q = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("field1", "a")), BooleanClause.Occur.MUST)
+        .add(new TermQuery(new Term("field1", "supercalifragilisticexpialidocious")), BooleanClause.Occur.MUST)
+        .add(new TermQuery(new Term("field1", "b")), BooleanClause.Occur.MUST)
+        .build();
+    Set<Term> expected
+        = Collections.singleton(new Term("field1", "supercalifragilisticexpialidocious"));
+    QueryTree tree = treeBuilder.buildTree(q, TermWeightor.DEFAULT);
+    Set<Term> terms = new HashSet<>();
+    tree.collectTerms((f, b) -> terms.add(new Term(f, b)));
+    assertEquals(expected, terms);
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestBooleanTermExtractor.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestBooleanTermExtractor.java
new file mode 100644
index 0000000..98c5a9a
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestBooleanTermExtractor.java
@@ -0,0 +1,135 @@
+/*
+ * 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.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestBooleanTermExtractor extends LuceneTestCase {
+
+  private static final QueryAnalyzer treeBuilder = new QueryAnalyzer();
+  private static final TermWeightor WEIGHTOR = TermWeightor.DEFAULT;
+
+  private Set<Term> collectTerms(Query query) {
+    Set<Term> terms = new HashSet<>();
+    QueryTree tree = treeBuilder.buildTree(query, TermWeightor.DEFAULT);
+    tree.collectTerms((f, b) -> terms.add(new Term(f, b)));
+    return terms;
+  }
+
+  public void testAllDisjunctionQueriesAreIncluded() {
+
+    Query bq = MonitorTestBase.parse("field1:term1 field1:term2");
+    Set<Term> terms = collectTerms(bq);
+    Set<Term> expected = new HashSet<>(Arrays.asList(
+        new Term("field1", "term1"),
+        new Term("field1", "term2")));
+    assertEquals(expected, terms);
+
+  }
+
+  public void testAllNestedDisjunctionClausesAreIncluded() {
+    Query q = MonitorTestBase.parse("field1:term3 (field1:term1 field1:term2)");
+    assertEquals(3, collectTerms(q).size());
+  }
+
+  public void testAllDisjunctionClausesOfAConjunctionAreExtracted() {
+    Query q = MonitorTestBase.parse("+(field1:term1 field1:term2) field1:term3");
+    assertEquals(2, collectTerms(q).size());
+  }
+
+  public void testConjunctionsOutweighDisjunctions() {
+    Query bq = MonitorTestBase.parse("field1:term1 +field1:term2");
+    Set<Term> expected = Collections.singleton(new Term("field1", "term2"));
+    assertEquals(expected, collectTerms(bq));
+  }
+
+  public void testDisjunctionsWithPureNegativeClausesReturnANYTOKEN() {
+    Query q = MonitorTestBase.parse("+field1:term1 +(field2:term22 (-field2:notterm))");
+    Set<Term> expected = Collections.singleton(new Term("field1", "term1"));
+    assertEquals(expected, collectTerms(q));
+  }
+
+  public void testDisjunctionsWithMatchAllNegativeClausesReturnANYTOKEN() {
+    Query q = MonitorTestBase.parse("+field1:term1 +(field2:term22 (*:* -field2:notterm))");
+    Set<Term> expected = Collections.singleton(new Term("field1", "term1"));
+    assertEquals(expected, collectTerms(q));
+  }
+
+  public void testMatchAllDocsIsOnlyQuery() {
+    // Set up - single MatchAllDocsQuery clause in a BooleanQuery
+    Query q = MonitorTestBase.parse("+*:*");
+    assertTrue(q instanceof BooleanQuery);
+    BooleanClause clause = ((BooleanQuery)q).iterator().next();
+    assertTrue(clause.getQuery() instanceof MatchAllDocsQuery);
+    assertEquals(BooleanClause.Occur.MUST, clause.getOccur());
+
+    Set<Term> terms = collectTerms(q);
+    assertEquals(1, terms.size());
+    Term t = terms.iterator().next();
+    assertEquals(TermFilteredPresearcher.ANYTOKEN_FIELD, t.field());
+  }
+
+  public void testMatchAllDocsMustWithKeywordShould() {
+    Query q = MonitorTestBase.parse("+*:* field1:term1");
+    // Because field1:term1 is optional, only the MatchAllDocsQuery is collected.
+    Set<Term> terms = collectTerms(q);
+    assertEquals(1, terms.size());
+    Term t = terms.iterator().next();
+    assertEquals(TermFilteredPresearcher.ANYTOKEN_FIELD, t.field());
+  }
+
+  public void testMatchAllDocsMustWithKeywordNot() throws Exception {
+    Query q = MonitorTestBase.parse("+*:* -field1:notterm");
+
+    // Because field1:notterm is negated, only the mandatory MatchAllDocsQuery is collected.
+    Set<Term> terms = collectTerms(q);
+    assertEquals(1, terms.size());
+    Term t = terms.iterator().next();
+    assertEquals(TermFilteredPresearcher.ANYTOKEN_FIELD, t.field());
+  }
+
+  public void testMatchAllDocsMustWithKeywordShouldAndKeywordNot() throws Exception {
+    Query q = MonitorTestBase.parse("+*:* field1:term1 -field2:notterm");
+
+    // Because field1:notterm is negated and field1:term1 is optional, only the mandatory MatchAllDocsQuery is collected.
+    Set<Term> terms = collectTerms(q);
+    assertEquals(1, terms.size());
+    Term t = terms.iterator().next();
+    assertEquals(TermFilteredPresearcher.ANYTOKEN_FIELD, t.field());
+  }
+
+  public void testMatchAllDocsMustAndOtherMustWithKeywordShouldAndKeywordNot() throws Exception {
+    Query q = MonitorTestBase.parse("+*:* +field9:term9 field1:term1 -field2:notterm");
+
+    // The queryterm collected by weight is the non-anynode, so field9:term9 shows up before MatchAllDocsQuery.
+    Set<Term> terms = collectTerms(q);
+    Set<Term> expected = Collections.singleton(new Term("field9", "term9"));
+    assertEquals(expected, terms);
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestCachePurging.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestCachePurging.java
new file mode 100644
index 0000000..8291e48
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestCachePurging.java
@@ -0,0 +1,153 @@
+/*
+ * 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.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.util.NamedThreadFactory;
+
+import static org.hamcrest.core.Is.is;
+
+public class TestCachePurging extends MonitorTestBase {
+
+  public void testQueryCacheCanBePurged() throws IOException {
+
+    final AtomicInteger purgeCount = new AtomicInteger();
+    MonitorUpdateListener listener = new MonitorUpdateListener() {
+      @Override
+      public void onPurge() {
+        purgeCount.incrementAndGet();
+      }
+    };
+
+    try (Monitor monitor = new Monitor(ANALYZER)) {
+      MonitorQuery[] queries = new MonitorQuery[]{
+          new MonitorQuery("1", parse("test1 test4")),
+          new MonitorQuery("2", parse("test2")),
+          new MonitorQuery("3", parse("test3"))
+      };
+      monitor.addQueryIndexUpdateListener(listener);
+      monitor.register(queries);
+      assertThat(monitor.getQueryCount(), is(3));
+      assertThat(monitor.getDisjunctCount(), is(4));
+      assertThat(monitor.getQueryCacheStats().cachedQueries, is(4));
+
+      Document doc = new Document();
+      doc.add(newTextField("field", "test1 test2 test3", Field.Store.NO));
+      assertThat(monitor.match(doc, QueryMatch.SIMPLE_MATCHER).getMatchCount(), is(3));
+
+      monitor.deleteById("1");
+      assertThat(monitor.getQueryCount(), is(2));
+      assertThat(monitor.getQueryCacheStats().cachedQueries, is(4));
+      assertThat(monitor.match(doc, QueryMatch.SIMPLE_MATCHER).getMatchCount(), is(2));
+
+      monitor.purgeCache();
+      assertThat(monitor.getQueryCacheStats().cachedQueries, is(2));
+
+      MatchingQueries<QueryMatch> result = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertThat(result.getMatchCount(), is(2));
+      assertTrue(purgeCount.get() > 0);
+    }
+  }
+
+  public void testConcurrentPurges() throws Exception {
+    int iters = Integer.getInteger("purgeIters", 2);
+    for (int i = 0; i < iters; i++) {
+      doConcurrentPurgesAndUpdatesTest();
+    }
+  }
+
+  private static void doConcurrentPurgesAndUpdatesTest() throws Exception {
+
+    final CountDownLatch startUpdating = new CountDownLatch(1);
+    final CountDownLatch finishUpdating = new CountDownLatch(1);
+
+    try (final Monitor monitor = new Monitor(ANALYZER)) {
+      Runnable updaterThread = () -> {
+        try {
+          startUpdating.await();
+          for (int i = 200; i < 400; i++) {
+            monitor.register(newMonitorQuery(i));
+          }
+          finishUpdating.countDown();
+        } catch (Exception e) {
+          throw new RuntimeException(e);
+        }
+      };
+
+      ExecutorService executor = Executors.newFixedThreadPool(1, new NamedThreadFactory("updaters"));
+      try {
+        executor.submit(updaterThread);
+
+        for (int i = 0; i < 200; i++) {
+          monitor.register(newMonitorQuery(i));
+        }
+        for (int i = 20; i < 80; i++) {
+          monitor.deleteById(Integer.toString(i));
+        }
+
+        assertEquals(200, monitor.getQueryCacheStats().cachedQueries);
+
+        startUpdating.countDown();
+        monitor.purgeCache();
+        finishUpdating.await();
+
+        assertEquals(340, monitor.getQueryCacheStats().cachedQueries);
+        Document doc = new Document();
+        doc.add(newTextField("field", "test", Field.Store.NO));
+        MatchingQueries<QueryMatch> matcher = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
+        assertEquals(0, matcher.getErrors().size());
+        assertEquals(340, matcher.getMatchCount());
+      } finally {
+        executor.shutdownNow();
+      }
+    }
+  }
+
+  private static MonitorQuery newMonitorQuery(int id) {
+    return new MonitorQuery(Integer.toString(id), parse("+test " + id));
+  }
+
+  public void testBackgroundPurges() throws IOException, InterruptedException {
+
+    MonitorConfiguration config = new MonitorConfiguration().setPurgeFrequency(1, TimeUnit.SECONDS);
+    try (Monitor monitor = new Monitor(ANALYZER, Presearcher.NO_FILTERING, config)) {
+
+      assertEquals(-1, monitor.getQueryCacheStats().lastPurged);
+
+      for (int i = 0; i < 100; i++) {
+        monitor.register(newMonitorQuery(i));
+      }
+      monitor.deleteById("5");
+      assertEquals(99, monitor.getQueryCacheStats().queries);
+      assertEquals(100, monitor.getQueryCacheStats().cachedQueries);
+
+      TimeUnit.SECONDS.sleep(2);
+      assertEquals(99, monitor.getQueryCacheStats().queries);
+      assertEquals(99, monitor.getQueryCacheStats().cachedQueries);
+      assertTrue(monitor.getQueryCacheStats().lastPurged > 0);
+    }
+  }
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestConcurrentQueryLoader.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestConcurrentQueryLoader.java
new file mode 100644
index 0000000..44b9fd1
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestConcurrentQueryLoader.java
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+public class TestConcurrentQueryLoader extends MonitorTestBase {
+
+  public void testLoading() throws Exception {
+    try (Monitor monitor = newMonitor()) {
+      try (ConcurrentQueryLoader loader = new ConcurrentQueryLoader(monitor)) {
+        for (int i = 0; i < 2000; i++) {
+          loader.add(new MonitorQuery(Integer.toString(i), parse("\"test " + i + "\"")));
+        }
+      }
+      assertEquals(2000, monitor.getQueryCount());
+    }
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestExplainingMatcher.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestExplainingMatcher.java
new file mode 100644
index 0000000..7362f95
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestExplainingMatcher.java
@@ -0,0 +1,53 @@
+/*
+ * 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.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.search.Explanation;
+
+public class TestExplainingMatcher extends MonitorTestBase {
+
+  public void testExplainingMatcher() throws IOException {
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", parse("test")), new MonitorQuery("2", parse("wibble")));
+
+      Document doc = new Document();
+      doc.add(newTextField("field", "test", Field.Store.NO));
+
+      MatchingQueries<ExplainingMatch> matches = monitor.match(doc, ExplainingMatch.MATCHER);
+      assertNotNull(matches.matches("1"));
+      assertNotNull(matches.matches("1").getExplanation());
+    }
+  }
+
+  public void testHashcodeAndEquals() {
+
+    ExplainingMatch m1 = new ExplainingMatch("1", Explanation.match(0.1f, "an explanation"));
+    ExplainingMatch m3 = new ExplainingMatch("1", Explanation.match(0.1f, "another explanation"));
+    ExplainingMatch m4 = new ExplainingMatch("1", Explanation.match(0.1f, "an explanation"));
+
+    assertEquals(m1, m4);
+    assertEquals(m1.hashCode(), m4.hashCode());
+    assertNotEquals(m1, m3);
+    assertNotEquals(m3, m4);
+  }
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestExtractors.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestExtractors.java
new file mode 100644
index 0000000..1b2ac8c
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestExtractors.java
@@ -0,0 +1,103 @@
+/*
+ * 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.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.BoostQuery;
+import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.DisjunctionMaxQuery;
+import org.apache.lucene.search.PhraseQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestExtractors extends LuceneTestCase {
+
+  private static final QueryAnalyzer treeBuilder = new QueryAnalyzer();
+
+  private Set<Term> collectTerms(Query query) {
+    Set<Term> terms = new HashSet<>();
+    QueryTree tree = treeBuilder.buildTree(query, TermWeightor.DEFAULT);
+    tree.collectTerms((f, b) -> terms.add(new Term(f, b)));
+    return terms;
+  }
+
+  public void testConstantScoreQueryExtractor() {
+
+    BooleanQuery.Builder bq = new BooleanQuery.Builder();
+    bq.add(new TermQuery(new Term("f", "q1")), BooleanClause.Occur.MUST);
+    bq.add(new TermQuery(new Term("f", "q2")), BooleanClause.Occur.SHOULD);
+
+    Query csqWithQuery = new ConstantScoreQuery(bq.build());
+    Set<Term> expected = Collections.singleton(new Term("f", "q1"));
+    assertEquals(expected, collectTerms(csqWithQuery));
+
+  }
+
+  public void testPhraseQueryExtractor() {
+
+    PhraseQuery.Builder pq = new PhraseQuery.Builder();
+    pq.add(new Term("f", "hello"));
+    pq.add(new Term("f", "encyclopedia"));
+
+    Set<Term> expected = Collections.singleton(new Term("f", "encyclopedia"));
+    assertEquals(expected, collectTerms(pq.build()));
+
+  }
+
+  public void testBoostQueryExtractor() {
+
+    BooleanQuery.Builder bq = new BooleanQuery.Builder();
+    bq.add(new TermQuery(new Term("f", "q1")), BooleanClause.Occur.MUST);
+    bq.add(new TermQuery(new Term("f", "q2")), BooleanClause.Occur.SHOULD);
+
+    Query boostQuery = new BoostQuery(bq.build(), 0.5f);
+    Set<Term> expected = Collections.singleton(new Term("f", "q1"));
+    assertEquals(expected, collectTerms(boostQuery));
+  }
+
+  public void testDisjunctionMaxExtractor() {
+
+    Query query = new DisjunctionMaxQuery(
+        Arrays.asList(new TermQuery(new Term("f", "t1")), new TermQuery(new Term("f", "t2"))), 0.1f
+    );
+    Set<Term> expected = new HashSet<>(Arrays.asList(
+        new Term("f", "t1"),
+        new Term("f", "t2")
+    ));
+    assertEquals(expected, collectTerms(query));
+  }
+
+  public void testBooleanExtractsFilter() {
+    Query q = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term("f", "must")), BooleanClause.Occur.MUST)
+        .add(new TermQuery(new Term("f", "filter")), BooleanClause.Occur.FILTER)
+        .build();
+    Set<Term> expected = Collections.singleton(new Term("f", "filter")); // it's longer, so it wins
+    assertEquals(expected, collectTerms(q));
+  }
+
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestFieldFilteredMultipassPresearcher.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestFieldFilteredMultipassPresearcher.java
new file mode 100644
index 0000000..75e708e
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestFieldFilteredMultipassPresearcher.java
@@ -0,0 +1,29 @@
+/*
+ * 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.util.Collections;
+
+public class TestFieldFilteredMultipassPresearcher extends FieldFilterPresearcherComponentTestBase {
+
+  @Override
+  protected Presearcher createPresearcher() {
+    return new MultipassTermFilteredPresearcher(2, 0, TermWeightor.DEFAULT,
+        Collections.emptyList(), Collections.singleton("language"));
+  }
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestFieldTermFilteredPresearcher.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestFieldTermFilteredPresearcher.java
new file mode 100644
index 0000000..38518ee
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestFieldTermFilteredPresearcher.java
@@ -0,0 +1,28 @@
+/*
+ * 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.util.Collections;
+
+public class TestFieldTermFilteredPresearcher extends FieldFilterPresearcherComponentTestBase {
+
+  @Override
+  protected Presearcher createPresearcher() {
+    return new TermFilteredPresearcher(TermWeightor.DEFAULT, Collections.emptyList(), Collections.singleton("language"));
+  }
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestForceNoBulkScoringQuery.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestForceNoBulkScoringQuery.java
new file mode 100644
index 0000000..7a22972
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestForceNoBulkScoringQuery.java
@@ -0,0 +1,82 @@
+/*
+ * 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.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.TextField;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.PrefixQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.ByteBuffersDirectory;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestForceNoBulkScoringQuery extends LuceneTestCase {
+
+  public void testEquality() {
+
+    TermQuery tq1 = new TermQuery(new Term("f", "t"));
+    TermQuery tq2 = new TermQuery(new Term("f", "t2"));
+    TermQuery tq3 = new TermQuery(new Term("f", "t2"));
+
+    assertEquals(new ForceNoBulkScoringQuery(tq1), new ForceNoBulkScoringQuery(tq1));
+    assertNotEquals(new ForceNoBulkScoringQuery(tq1), new ForceNoBulkScoringQuery(tq2));
+    assertEquals(new ForceNoBulkScoringQuery(tq2), new ForceNoBulkScoringQuery(tq3));
+
+    assertEquals(new ForceNoBulkScoringQuery(tq2).hashCode(), new ForceNoBulkScoringQuery(tq3).hashCode());
+  }
+
+  public void testRewrite() throws IOException {
+
+    try (Directory dir = new ByteBuffersDirectory();
+         IndexWriter iw = new IndexWriter(dir, new IndexWriterConfig(new StandardAnalyzer()))) {
+
+      Document doc = new Document();
+      doc.add(new TextField("field", "term1 term2 term3 term4", Field.Store.NO));
+      iw.addDocument(doc);
+      iw.commit();
+
+      IndexReader reader = DirectoryReader.open(dir);
+
+      PrefixQuery pq = new PrefixQuery(new Term("field", "term"));
+      ForceNoBulkScoringQuery q = new ForceNoBulkScoringQuery(pq);
+
+      assertEquals(q.getWrappedQuery(), pq);
+
+      Query rewritten = q.rewrite(reader);
+      assertTrue(rewritten instanceof ForceNoBulkScoringQuery);
+
+      Query inner = ((ForceNoBulkScoringQuery) rewritten).getWrappedQuery();
+      assertNotEquals(inner, pq);
+
+
+    }
+
+
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestHighlightingMatcher.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestHighlightingMatcher.java
new file mode 100644
index 0000000..eb13db4
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestHighlightingMatcher.java
@@ -0,0 +1,527 @@
+/*
+ * 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.Arrays;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.analysis.core.WhitespaceTokenizer;
+import org.apache.lucene.analysis.standard.StandardAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.queryparser.complexPhrase.ComplexPhraseQueryParser;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.DisjunctionMaxQuery;
+import org.apache.lucene.search.PrefixQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.RegexpQuery;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.WildcardQuery;
+import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper;
+import org.apache.lucene.search.spans.SpanNearQuery;
+import org.apache.lucene.search.spans.SpanQuery;
+import org.apache.lucene.search.spans.SpanTermQuery;
+
+
+public class TestHighlightingMatcher extends MonitorTestBase {
+
+  private static final Analyzer WHITESPACE = new WhitespaceAnalyzer();
+
+  public static Document buildDoc(String text) {
+    Document doc = new Document();
+    doc.add(newTextField(FIELD, text, Field.Store.NO));
+    return doc;
+  }
+
+  public void testSingleTermQueryMatchesSingleDocument() throws IOException {
+
+    try (Monitor monitor = newMonitor()) {
+      MonitorQuery mq = new MonitorQuery("query1", parse("test"));
+      monitor.register(mq);
+
+      MatchingQueries<HighlightsMatch> matches = monitor.match(buildDoc("this is a test document"),
+          HighlightsMatch.MATCHER);
+      assertEquals(1, matches.getMatchCount());
+      HighlightsMatch match = matches.matches("query1");
+      assertTrue(match.getHits(FIELD).contains(new HighlightsMatch.Hit(3, 10, 3, 14)));
+    }
+  }
+
+  public void testSinglePhraseQueryMatchesSingleDocument() throws IOException {
+
+    try (Monitor monitor = newMonitor()) {
+      MonitorQuery mq = new MonitorQuery("query1", parse("\"test document\""));
+      monitor.register(mq);
+
+      MatchingQueries<HighlightsMatch> matches = monitor.match(buildDoc("this is a test document"),
+          HighlightsMatch.MATCHER);
+      assertEquals(1, matches.getMatchCount());
+      HighlightsMatch m = matches.matches("query1");
+      assertTrue(m.getHits(FIELD).contains(new HighlightsMatch.Hit(3, 10, 4, 23)));
+    }
+
+  }
+
+  public void testToString() {
+
+    HighlightsMatch match = new HighlightsMatch("1");
+    match.addHit("field", 2, 3, -1, -1);
+    match.addHit("field", 0, 1, -1, -1);
+    match.addHit("afield", 0, 1, 0, 4);
+
+    assertEquals("Match(query=1){hits={afield=[0(0)->1(4)], field=[0(-1)->1(-1), 2(-1)->3(-1)]}}", match.toString());
+  }
+
+  public void testMultiFieldQueryMatches() throws IOException {
+
+    Document doc = new Document();
+    doc.add(newTextField("field1", "this is a test of field one", Field.Store.NO));
+    doc.add(newTextField("field2", "and this is an additional test", Field.Store.NO));
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("query1", parse("field1:test field2:test")));
+
+      MatchingQueries<HighlightsMatch> matches = monitor.match(doc, HighlightsMatch.MATCHER);
+      assertEquals(1, matches.getMatchCount());
+
+      HighlightsMatch m = matches.matches("query1");
+      assertNotNull(m);
+      assertTrue(m.getFields().contains("field1"));
+      assertTrue(m.getHits("field1").contains(new HighlightsMatch.Hit(3, 10, 3, 14)));
+      assertTrue(m.getHits("field2").contains(new HighlightsMatch.Hit(5, 26, 5, 30)));
+    }
+
+  }
+
+  public void testQueryErrors() throws IOException {
+
+    try (Monitor monitor = new Monitor(ANALYZER, Presearcher.NO_FILTERING)) {
+
+      monitor.register(new MonitorQuery("1", parse("test")),
+          new MonitorQuery("2", new ThrowOnRewriteQuery()),
+          new MonitorQuery("3", parse("document")),
+          new MonitorQuery("4", parse("foo")));
+
+      MatchingQueries<HighlightsMatch> matches = monitor.match(buildDoc("this is a test document"), HighlightsMatch.MATCHER);
+      assertEquals(4, matches.getQueriesRun());
+      assertEquals(2, matches.getMatchCount());
+      assertEquals(1, matches.getErrors().size());
+    }
+  }
+
+  public void testWildcards() throws IOException {
+
+    try (Monitor monitor = newMonitor()) {
+
+      monitor.register(new MonitorQuery("1", new RegexpQuery(new Term(FIELD, "he.*"))));
+
+      MatchingQueries<HighlightsMatch> matches = monitor.match(buildDoc("hello world"), HighlightsMatch.MATCHER);
+      assertEquals(1, matches.getQueriesRun());
+      assertEquals(1, matches.getMatchCount());
+      assertEquals(1, matches.matches("1").getHitCount());
+    }
+  }
+
+  public void testWildcardCombinations() throws Exception {
+
+    final BooleanQuery bq = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term(FIELD, "term1")), BooleanClause.Occur.MUST)
+        .add(new PrefixQuery(new Term(FIELD, "term2")), BooleanClause.Occur.MUST)
+        .add(new TermQuery(new Term(FIELD, "term3")), BooleanClause.Occur.MUST_NOT)
+        .build();
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", bq));
+
+      MatchingQueries<HighlightsMatch> matches = monitor.match(buildDoc("term1 term22 term4"), HighlightsMatch.MATCHER);
+      HighlightsMatch m = matches.matches("1");
+      assertNotNull(m);
+      assertEquals(2, m.getHitCount());
+    }
+
+  }
+
+  public void testDisjunctionMaxQuery() throws IOException {
+    final DisjunctionMaxQuery query = new DisjunctionMaxQuery(Arrays.asList(
+        new TermQuery(new Term(FIELD, "term1")), new PrefixQuery(new Term(FIELD, "term2"))
+    ), 1.0f);
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", query));
+      MatchingQueries<HighlightsMatch> matches = monitor.match(buildDoc("term1 term2 term3"), HighlightsMatch.MATCHER);
+      HighlightsMatch m = matches.matches("1");
+      assertNotNull(m);
+      assertEquals(2, m.getHitCount());
+    }
+
+  }
+
+  public void testIdenticalMatches() throws Exception {
+
+    final BooleanQuery bq = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term(FIELD, "term1")), BooleanClause.Occur.MUST)
+        .add(new TermQuery(new Term(FIELD, "term1")), BooleanClause.Occur.SHOULD)
+        .build();
+
+    try (Monitor monitor = new Monitor(ANALYZER)) {
+      monitor.register(new MonitorQuery("1", bq));
+      MatchingQueries<HighlightsMatch> matches = monitor.match(buildDoc("term1 term2"), HighlightsMatch.MATCHER);
+      HighlightsMatch m = matches.matches("1");
+      assertNotNull(m);
+      assertEquals(1, m.getHitCount());
+    }
+
+  }
+
+  public void testWildcardBooleanRewrites() throws Exception {
+
+    final Query wc = new PrefixQuery(new Term(FIELD, "term1"));
+
+    final Query wrapper = new BooleanQuery.Builder()
+        .add(wc, BooleanClause.Occur.MUST)
+        .build();
+
+    final Query wrapper2 = new BooleanQuery.Builder()
+        .add(wrapper, BooleanClause.Occur.MUST)
+        .build();
+
+    final BooleanQuery bq = new BooleanQuery.Builder()
+        .add(new PrefixQuery(new Term(FIELD, "term2")), BooleanClause.Occur.MUST)
+        .add(wrapper2, BooleanClause.Occur.MUST_NOT)
+        .build();
+
+    try (Monitor monitor = new Monitor(ANALYZER)) {
+
+      monitor.register(new MonitorQuery("1", bq));
+      MatchingQueries<HighlightsMatch> matches = monitor.match(buildDoc("term2 term"), HighlightsMatch.MATCHER);
+      HighlightsMatch m = matches.matches("1");
+      assertNotNull(m);
+      assertEquals(1, m.getHitCount());
+
+      matches = monitor.match(buildDoc("term2 term"), HighlightsMatch.MATCHER);
+      m = matches.matches("1");
+      assertNotNull(m);
+      assertEquals(1, m.getHitCount());
+    }
+  }
+
+  public void testWildcardProximityRewrites() throws Exception {
+    final SpanNearQuery snq = SpanNearQuery.newOrderedNearQuery(FIELD)
+        .addClause(new SpanMultiTermQueryWrapper<>(new WildcardQuery(new Term(FIELD, "term*"))))
+        .addClause(new SpanTermQuery(new Term(FIELD, "foo")))
+        .build();
+
+    try (Monitor monitor = newMonitor()) {
+
+      monitor.register(new MonitorQuery("1", snq));
+
+      MatchingQueries<HighlightsMatch> matches = monitor.match(buildDoc("term1 foo"), HighlightsMatch.MATCHER);
+      HighlightsMatch m = matches.matches("1");
+      assertNotNull(m);
+      assertEquals(2, m.getHitCount());
+    }
+  }
+
+  public void testDisjunctionWithOrderedNearSpans() throws Exception {
+
+    final Query bq = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term(FIELD, "a")), BooleanClause.Occur.SHOULD)
+        .add(SpanNearQuery.newOrderedNearQuery(FIELD)
+            .addClause(new SpanTermQuery(new Term(FIELD, "b")))
+            .addClause(new SpanTermQuery(new Term(FIELD, "c")))
+            .setSlop(1)
+            .build(), BooleanClause.Occur.SHOULD)
+        .build();
+    final Query parent = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term(FIELD, "a")), BooleanClause.Occur.MUST)
+        .add(bq, BooleanClause.Occur.MUST)
+        .build();
+
+    try (Monitor monitor = new Monitor(ANALYZER)) {
+      monitor.register(new MonitorQuery("1", parent));
+
+      Document doc = buildDoc("a b x x x x c");
+      MatchingQueries<HighlightsMatch> matches = monitor.match(doc, HighlightsMatch.MATCHER);
+
+      HighlightsMatch m = matches.matches("1");
+      assertNotNull(m);
+      assertEquals(1, m.getHitCount());
+    }
+
+  }
+
+  public void testDisjunctionWithUnorderedNearSpans() throws Exception {
+
+    final Query bq = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term(FIELD, "a")), BooleanClause.Occur.SHOULD)
+        .add(SpanNearQuery.newUnorderedNearQuery(FIELD)
+            .addClause(new SpanTermQuery(new Term(FIELD, "b")))
+            .addClause(new SpanTermQuery(new Term(FIELD, "c")))
+            .setSlop(1)
+            .build(), BooleanClause.Occur.SHOULD)
+        .build();
+    final Query parent = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term(FIELD, "a")), BooleanClause.Occur.MUST)
+        .add(bq, BooleanClause.Occur.MUST)
+        .build();
+
+    try (Monitor monitor = new Monitor(ANALYZER)) {
+      monitor.register(new MonitorQuery("1", parent));
+
+      Document doc = buildDoc("a b x x x x c");
+      MatchingQueries<HighlightsMatch> matches = monitor.match(doc, HighlightsMatch.MATCHER);
+
+      HighlightsMatch m = matches.matches("1");
+      assertNotNull(m);
+      assertEquals(1, m.getHitCount());
+    }
+
+  }
+
+  public void testEquality() {
+
+    HighlightsMatch m1 = new HighlightsMatch("1");
+    m1.addHit("field", 0, 1, 0, 1);
+
+    HighlightsMatch m2 = new HighlightsMatch("1");
+    m2.addHit("field", 0, 1, 0, 1);
+
+    HighlightsMatch m3 = new HighlightsMatch("1");
+    m3.addHit("field", 0, 2, 0, 1);
+
+    HighlightsMatch m4 = new HighlightsMatch("2");
+    m4.addHit("field", 0, 1, 0, 1);
+
+    assertEquals(m1, m2);
+    assertEquals(m1.hashCode(), m2.hashCode());
+    assertNotEquals(m1, m3);
+    assertNotEquals(m1, m4);
+  }
+
+  public void testMutliValuedFieldWithNonDefaultGaps() throws IOException {
+
+    Analyzer analyzer = new Analyzer() {
+      @Override
+      public int getPositionIncrementGap(String fieldName) {
+        return 1000;
+      }
+
+      @Override
+      public int getOffsetGap(String fieldName) {
+        return 2000;
+      }
+
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName) {
+        return new TokenStreamComponents(new WhitespaceTokenizer());
+      }
+    };
+
+    MonitorQuery mq = new MonitorQuery("query", parse(FIELD + ":\"hello world\"~5"));
+    try (Monitor monitor = newMonitor(analyzer)) {
+      monitor.register(mq);
+
+      Document doc1 = new Document();
+      doc1.add(newTextField(FIELD, "hello world", Field.Store.NO));
+      doc1.add(newTextField(FIELD, "goodbye", Field.Store.NO));
+      MatchingQueries<HighlightsMatch> matcher1 = monitor.match(doc1, HighlightsMatch.MATCHER);
+      assertEquals(1, matcher1.getMatchCount());
+      HighlightsMatch m1 = matcher1.matches("query");
+      assertNotNull(m1);
+      assertTrue(m1.getFields().contains(FIELD));
+      assertTrue(m1.getHits(FIELD).contains(new HighlightsMatch.Hit(0, 0, 1, 11)));
+
+      Document doc2 = new Document();
+      doc1.add(newTextField(FIELD, "hello", Field.Store.NO));
+      doc1.add(newTextField(FIELD, "world", Field.Store.NO));
+      MatchingQueries<HighlightsMatch> matcher2 = monitor.match(doc2, HighlightsMatch.MATCHER);
+      assertNull(matcher2.matches("query"));
+      assertEquals(0, matcher2.getMatchCount());
+
+      Document doc3 = new Document();
+      doc3.add(newTextField(FIELD, "hello world", Field.Store.NO));
+      doc3.add(newTextField(FIELD, "hello goodbye world", Field.Store.NO));
+      MatchingQueries<HighlightsMatch> matcher3 = monitor.match(doc3, HighlightsMatch.MATCHER);
+      assertEquals(1, matcher3.getMatchCount());
+      HighlightsMatch m3 = matcher3.matches("query");
+      assertNotNull(m3);
+      assertTrue(m3.getFields().contains(FIELD));
+      assertTrue(m3.getHits(FIELD).contains(new HighlightsMatch.Hit(0, 0, 1, 11)));
+      assertTrue(m3.getHits(FIELD).contains(new HighlightsMatch.Hit(1002, 2011, 1004, 2030)));
+    }
+
+  }
+
+  public void testDisjunctionWithOrderedNearMatch() throws Exception {
+
+    final Query bq = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term(FIELD, "a")), BooleanClause.Occur.SHOULD)
+        .add(SpanNearQuery.newOrderedNearQuery(FIELD)
+            .addClause(new SpanTermQuery(new Term(FIELD, "b")))
+            .addClause(new SpanTermQuery(new Term(FIELD, "c")))
+            .setSlop(1)
+            .build(), BooleanClause.Occur.SHOULD)
+        .build();
+    final Query parent = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term(FIELD, "a")), BooleanClause.Occur.MUST)
+        .add(bq, BooleanClause.Occur.MUST)
+        .build();
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", parent));
+
+      Document doc = buildDoc("a b c");
+      MatchingQueries<HighlightsMatch> matches = monitor.match(doc, HighlightsMatch.MATCHER);
+
+      HighlightsMatch m = matches.matches("1");
+      assertNotNull(m);
+      assertEquals(3, m.getHitCount());
+      assertTrue(m.getFields().contains(FIELD));
+      assertTrue(m.getHits(FIELD).contains(new HighlightsMatch.Hit(0, 0, 0, 1)));
+      assertTrue(m.getHits(FIELD).contains(new HighlightsMatch.Hit(1, 2, 1, 3)));
+      assertTrue(m.getHits(FIELD).contains(new HighlightsMatch.Hit(2, 4, 2, 5)));
+    }
+
+  }
+
+  public void testUnorderedNearWithinOrderedNear() throws Exception {
+
+    final SpanQuery spanPhrase = SpanNearQuery.newOrderedNearQuery(FIELD)
+        .addClause(new SpanTermQuery(new Term(FIELD, "time")))
+        .addClause(new SpanTermQuery(new Term(FIELD, "men")))
+        .setSlop(1)
+        .build();
+
+    final SpanQuery unorderedNear = SpanNearQuery.newUnorderedNearQuery(FIELD)
+        .addClause(spanPhrase)
+        .addClause(new SpanTermQuery(new Term(FIELD, "all")))
+        .setSlop(5)
+        .build();
+
+    final SpanQuery orderedNear = SpanNearQuery.newOrderedNearQuery(FIELD)
+        .addClause(new SpanTermQuery(new Term(FIELD, "the")))
+        .addClause(unorderedNear)
+        .setSlop(10)
+        .build();
+
+    final Query innerConjunct = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term(FIELD, "is")), BooleanClause.Occur.MUST)
+        .add(orderedNear, BooleanClause.Occur.MUST)
+        .build();
+
+    final Query disjunct = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term(FIELD, "now")), BooleanClause.Occur.SHOULD)
+        .add(innerConjunct, BooleanClause.Occur.SHOULD)
+        .build();
+
+    final Query outerConjunct = new BooleanQuery.Builder()
+        .add(disjunct, BooleanClause.Occur.MUST)
+        .add(new TermQuery(new Term(FIELD, "good")), BooleanClause.Occur.MUST)
+        .build();
+
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", outerConjunct));
+      Document doc = buildDoc("now is the time for all good men");
+      MatchingQueries<HighlightsMatch> matches = monitor.match(doc, HighlightsMatch.MATCHER);
+      HighlightsMatch m = matches.matches("1");
+      assertEquals(2, m.getHitCount());
+      assertTrue(m.getFields().contains(FIELD));
+      assertTrue(m.getHits(FIELD).contains(new HighlightsMatch.Hit(0, 0, 0, 3)));
+      assertTrue(m.getHits(FIELD).contains(new HighlightsMatch.Hit(6, 24, 6, 28)));
+    }
+
+  }
+
+  public void testMinShouldMatchQuery() throws Exception {
+
+    final Query minq = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term(FIELD, "x")), BooleanClause.Occur.SHOULD)
+        .add(new TermQuery(new Term(FIELD, "y")), BooleanClause.Occur.SHOULD)
+        .add(new TermQuery(new Term(FIELD, "z")), BooleanClause.Occur.SHOULD)
+        .setMinimumNumberShouldMatch(2)
+        .build();
+
+    final Query bq = new BooleanQuery.Builder()
+        .add(new TermQuery(new Term(FIELD, "a")), BooleanClause.Occur.MUST)
+        .add(new TermQuery(new Term(FIELD, "b")), BooleanClause.Occur.MUST)
+        .add(minq, BooleanClause.Occur.SHOULD)
+        .build();
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", bq));
+      Document doc = buildDoc("a b x");
+      MatchingQueries<HighlightsMatch> matches = monitor.match(doc, HighlightsMatch.MATCHER);
+      HighlightsMatch m = matches.matches("1");
+      assertNotNull(m);
+      assertEquals(2, m.getHitCount());
+      assertTrue(m.getFields().contains(FIELD));
+      assertTrue(m.getHits(FIELD).contains(new HighlightsMatch.Hit(0, 0, 0, 1)));
+      assertTrue(m.getHits(FIELD).contains(new HighlightsMatch.Hit(1, 2, 1, 3)));
+    }
+
+  }
+
+  public void testComplexPhraseQueryParser() throws Exception {
+
+    ComplexPhraseQueryParser cpqp = new ComplexPhraseQueryParser(FIELD, new StandardAnalyzer());
+    Query query = cpqp.parse("\"x b\"");
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", query));
+      Document doc = buildDoc("x b c");
+      MatchingQueries<HighlightsMatch> matches = monitor.match(doc, HighlightsMatch.MATCHER);
+      HighlightsMatch m = matches.matches("1");
+      assertNotNull(m);
+      assertEquals(2, m.getHitCount());
+      assertTrue(m.getFields().contains(FIELD));
+    }
+
+  }
+
+  public void testHighlightBatches() throws Exception {
+    String query = "\"cell biology\"";
+
+    try (Monitor monitor = newMonitor(WHITESPACE)) {
+
+      monitor.register(new MonitorQuery("query0", parse("non matching query")));
+      monitor.register(new MonitorQuery("query1", parse(query)));
+      monitor.register(new MonitorQuery("query2", parse("biology")));
+
+      Document doc1 = new Document();
+      doc1.add(newTextField(FIELD, "the cell biology count", Field.Store.NO)); // matches
+      Document doc2 = new Document();
+      doc2.add(newTextField(FIELD, "nope", Field.Store.NO));
+      Document doc3 = new Document();
+      doc3.add(newTextField(FIELD, "biology text", Field.Store.NO));
+
+      MultiMatchingQueries<HighlightsMatch> matches = monitor.match(new Document[]{doc1, doc2, doc3}, HighlightsMatch.MATCHER);
+      assertEquals(2, matches.getMatchCount(0));
+      assertEquals(0, matches.getMatchCount(1));
+      assertEquals(1, matches.getMatchCount(2));
+      HighlightsMatch m1 = matches.matches("query1", 0);
+      assertTrue(m1.getHits(FIELD).contains(new HighlightsMatch.Hit(1, 4, 2, 16)));
+      HighlightsMatch m2 = matches.matches("query2", 2);
+      assertTrue(m2.getHits(FIELD).contains(new HighlightsMatch.Hit(0, 0, 0, 7)));
+    }
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestMatchAllPresearcher.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestMatchAllPresearcher.java
new file mode 100644
index 0000000..cf1bf41
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestMatchAllPresearcher.java
@@ -0,0 +1,27 @@
+/*
+ * 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;
+
+public class TestMatchAllPresearcher extends PresearcherTestBase {
+
+  @Override
+  protected Presearcher createPresearcher() {
+    return Presearcher.NO_FILTERING;
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestMonitor.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestMonitor.java
new file mode 100644
index 0000000..73443e1
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestMonitor.java
@@ -0,0 +1,245 @@
+/*
+ * 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.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.analysis.core.WhitespaceTokenizer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+
+public class TestMonitor extends MonitorTestBase {
+
+  private static final Analyzer ANALYZER = new WhitespaceAnalyzer();
+
+  public void testSingleTermQueryMatchesSingleDocument() throws IOException {
+
+    Document doc = new Document();
+    doc.add(newTextField(FIELD, "This is a test document", Field.Store.NO));
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("query1", new TermQuery(new Term(FIELD, "test"))));
+
+      MatchingQueries<QueryMatch> matches = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertNotNull(matches.getMatches());
+      assertEquals(1, matches.getMatchCount());
+      assertNotNull(matches.matches("query1"));
+    }
+  }
+
+  public void testMatchStatisticsAreReported() throws IOException {
+
+    Document doc = new Document();
+    doc.add(newTextField(FIELD, "This is a test document", Field.Store.NO));
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("query1", new TermQuery(new Term(MonitorTestBase.FIELD, "test"))));
+
+      MatchingQueries<QueryMatch> matches = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertEquals(1, matches.getQueriesRun());
+      assertTrue(matches.getQueryBuildTime() > -1);
+      assertTrue(matches.getSearchTime() > -1);
+    }
+  }
+
+  public void testUpdatesOverwriteOldQueries() throws IOException {
+
+    Document doc = new Document();
+    doc.add(newTextField(FIELD, "that", Field.Store.NO));
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("query1", new TermQuery(new Term(MonitorTestBase.FIELD, "this"))));
+      monitor.register(new MonitorQuery("query1", new TermQuery(new Term(MonitorTestBase.FIELD, "that"))));
+
+      MatchingQueries<QueryMatch> matches = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertNotNull(matches.matches("query1"));
+      assertEquals(1, matches.getQueriesRun());
+    }
+  }
+
+  public void testCanDeleteById() throws IOException {
+
+    Document doc = new Document();
+    doc.add(newTextField(FIELD, "other things", Field.Store.NO));
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("query1", new TermQuery(new Term(MonitorTestBase.FIELD, "this"))));
+      monitor.register(
+          new MonitorQuery("query2", new TermQuery(new Term(MonitorTestBase.FIELD, "that"))),
+          new MonitorQuery("query3", new TermQuery(new Term(MonitorTestBase.FIELD, "other"))));
+      assertEquals(3, monitor.getQueryCount());
+
+      monitor.deleteById("query2", "query1");
+      assertEquals(1, monitor.getQueryCount());
+
+      MatchingQueries<QueryMatch> matches = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertEquals(1, matches.getQueriesRun());
+      assertNotNull(matches.matches("query3"));
+    }
+
+  }
+
+  public void testCanClearTheMonitor() throws IOException {
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(
+          new MonitorQuery("query1", new MatchAllDocsQuery()),
+          new MonitorQuery("query2", new MatchAllDocsQuery()),
+          new MonitorQuery("query3", new MatchAllDocsQuery()));
+      assertEquals(3, monitor.getQueryCount());
+
+      monitor.clear();
+      assertEquals(0, monitor.getQueryCount());
+    }
+  }
+
+  public void testMatchesAgainstAnEmptyMonitor() throws IOException {
+
+    try (Monitor monitor = newMonitor()) {
+      assertEquals(0, monitor.getQueryCount());
+      Document doc = new Document();
+      doc.add(newTextField(FIELD, "This is a test document", Field.Store.NO));
+      MatchingQueries<QueryMatch> matches = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertEquals(0, matches.getQueriesRun());
+    }
+
+  }
+
+  public void testUpdateReporting() throws IOException {
+
+    List<MonitorQuery> queries = new ArrayList<>(10400);
+    for (int i = 0; i < 10355; i++) {
+      queries.add(new MonitorQuery(Integer.toString(i), MonitorTestBase.parse("test")));
+    }
+
+    final int[] expectedSizes = new int[]{5001, 5001, 353};
+    final AtomicInteger callCount = new AtomicInteger();
+    final AtomicInteger updateCount = new AtomicInteger();
+
+    MonitorUpdateListener listener = new MonitorUpdateListener() {
+
+      @Override
+      public void afterUpdate(List<MonitorQuery> updates) {
+        int calls = callCount.getAndIncrement();
+        updateCount.addAndGet(updates.size());
+        assertEquals(expectedSizes[calls], updates.size());
+      }
+    };
+
+    try (Monitor monitor = new Monitor(ANALYZER)) {
+      monitor.addQueryIndexUpdateListener(listener);
+      monitor.register(queries);
+      assertEquals(10355, updateCount.get());
+    }
+  }
+
+  public void testMatcherMetadata() throws IOException {
+    try (Monitor monitor = newMonitor()) {
+      HashMap<String, String> metadataMap = new HashMap<>();
+      metadataMap.put("key", "value");
+
+      monitor.register(new MonitorQuery(Integer.toString(1), MonitorTestBase.parse("+test " + 1), null, metadataMap));
+
+      Document doc = new Document();
+      doc.add(newTextField(FIELD, "This is a test document", Field.Store.NO));
+
+      MatcherFactory<QueryMatch> testMatcherFactory = docs -> new CandidateMatcher<QueryMatch>(docs) {
+        @Override
+        protected void matchQuery(String queryId, Query matchQuery, Map<String, String> metadata) {
+          assertEquals("value", metadata.get("key"));
+        }
+
+        @Override
+        public QueryMatch resolve(QueryMatch match1, QueryMatch match2) {
+          return null;
+        }
+      };
+
+      monitor.match(doc, testMatcherFactory);
+    }
+  }
+
+  public void testDocumentBatching() throws IOException {
+
+    Document doc1 = new Document();
+    doc1.add(newTextField(FIELD, "This is a test document", Field.Store.NO));
+    Document doc2 = new Document();
+    doc2.add(newTextField(FIELD, "This is a kangaroo document", Field.Store.NO));
+
+
+    try (Monitor monitor = new Monitor(ANALYZER)) {
+      monitor.register(new MonitorQuery("1", new TermQuery(new Term(MonitorTestBase.FIELD, "kangaroo"))));
+
+      MultiMatchingQueries<QueryMatch> response = monitor.match(new Document[]{ doc1, doc2 }, QueryMatch.SIMPLE_MATCHER);
+      assertEquals(2, response.getBatchSize());
+    }
+  }
+
+  public void testMutliValuedFieldWithNonDefaultGaps() throws IOException {
+
+    Analyzer analyzer = new Analyzer() {
+      @Override
+      public int getPositionIncrementGap(String fieldName) {
+        return 1000;
+      }
+
+      @Override
+      public int getOffsetGap(String fieldName) {
+        return 2000;
+      }
+
+      @Override
+      protected TokenStreamComponents createComponents(String fieldName) {
+        return new TokenStreamComponents(new WhitespaceTokenizer());
+      }
+    };
+
+    MonitorQuery mq = new MonitorQuery("query", MonitorTestBase.parse(MonitorTestBase.FIELD + ":\"hello world\"~5"));
+
+    try (Monitor monitor = new Monitor(analyzer)) {
+      monitor.register(mq);
+
+      Document doc1 = new Document();
+      doc1.add(newTextField(FIELD, "hello world", Field.Store.NO));
+      doc1.add(newTextField(FIELD, "goodbye", Field.Store.NO));
+
+      MatchingQueries<QueryMatch> matches = monitor.match(doc1, QueryMatch.SIMPLE_MATCHER);
+      assertNotNull(matches.getMatches());
+      assertEquals(1, matches.getMatchCount());
+      assertNotNull(matches.matches("query"));
+
+      Document doc2 = new Document();
+      doc2.add(newTextField(FIELD, "hello", Field.Store.NO));
+      doc2.add(newTextField(FIELD, "world", Field.Store.NO));
+      matches = monitor.match(doc2, QueryMatch.SIMPLE_MATCHER);
+      assertEquals(0, matches.getMatchCount());
+    }
+
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestMonitorErrorHandling.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestMonitorErrorHandling.java
new file mode 100644
index 0000000..5e2863c
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestMonitorErrorHandling.java
@@ -0,0 +1,57 @@
+/*
+ * 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.util.HashMap;
+import java.util.Map;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.search.MatchAllDocsQuery;
+
+public class TestMonitorErrorHandling extends MonitorTestBase {
+
+  public void testMonitorErrors() throws Exception {
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(
+          MonitorTestBase.mq("1", "test"),
+          new MonitorQuery("2", MonitorTestBase.parse("test")),
+          new MonitorQuery("3", new ThrowOnRewriteQuery()));
+
+      Document doc = new Document();
+      doc.add(newTextField(FIELD, "test", Field.Store.NO));
+      MatchingQueries<QueryMatch> matches = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
+
+      assertEquals(1, matches.getErrors().size());
+      assertEquals("Error rewriting", matches.getErrors().get("3").getMessage());
+      assertEquals(2, matches.getMatchCount());
+      assertEquals(3, matches.getQueriesRun());
+    }
+  }
+
+  public void testMonitorQueryNullValues() {
+    IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> {
+      Map<String, String> metadata2 = new HashMap<>();
+      metadata2.put("key", null);
+      new MonitorQuery("id", new MatchAllDocsQuery(), null, metadata2);
+    });
+    assertEquals("Null value for key key in metadata map", e.getMessage());
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestMonitorPersistence.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestMonitorPersistence.java
new file mode 100644
index 0000000..821ab50
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestMonitorPersistence.java
@@ -0,0 +1,72 @@
+/*
+ * 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.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.search.MatchAllDocsQuery;
+
+public class TestMonitorPersistence extends MonitorTestBase {
+
+  private Path indexDirectory = createTempDir();
+
+  public void testCacheIsRepopulated() throws IOException {
+
+    Document doc = new Document();
+    doc.add(newTextField(FIELD, "test", Field.Store.NO));
+    MonitorConfiguration config = new MonitorConfiguration()
+        .setIndexPath(indexDirectory, MonitorQuerySerializer.fromParser(MonitorTestBase::parse));
+
+    try (Monitor monitor = new Monitor(ANALYZER, config)) {
+      monitor.register(
+          mq("1", "test"),
+          mq("2", "test"),
+          mq("3", "test", "language", "en"),
+          mq("4", "test", "wibble", "quack"));
+
+      assertEquals(4, monitor.match(doc, QueryMatch.SIMPLE_MATCHER).getMatchCount());
+
+      IllegalArgumentException e = expectThrows(IllegalArgumentException.class,
+          () -> monitor.register(new MonitorQuery("5", new MatchAllDocsQuery(), null, Collections.emptyMap())));
+      assertEquals("Cannot add a MonitorQuery with a null string representation to a non-ephemeral Monitor", e.getMessage());
+    }
+
+    try (Monitor monitor2 = new Monitor(ANALYZER, config)) {
+      assertEquals(4, monitor2.getQueryCount());
+      assertEquals(4, monitor2.match(doc, QueryMatch.SIMPLE_MATCHER).getMatchCount());
+
+      MonitorQuery mq = monitor2.getQuery("4");
+      assertEquals("quack", mq.getMetadata().get("wibble"));
+    }
+
+  }
+
+  public void testEphemeralMonitorDoesNotStoreQueries() throws IOException {
+
+    try (Monitor monitor2 = new Monitor(ANALYZER)) {
+      IllegalStateException e = expectThrows(IllegalStateException.class, () -> monitor2.getQuery("query"));
+      assertEquals("Cannot get queries from an index with no MonitorQuerySerializer", e.getMessage());
+    }
+
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestMultipassPresearcher.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestMultipassPresearcher.java
new file mode 100644
index 0000000..18afe73
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestMultipassPresearcher.java
@@ -0,0 +1,123 @@
+/*
+ * 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.analysis.core.KeywordAnalyzer;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.memory.MemoryIndex;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.TermInSetQuery;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.ByteBuffersDirectory;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+
+public class TestMultipassPresearcher extends PresearcherTestBase {
+
+  @Override
+  protected Presearcher createPresearcher() {
+    return new MultipassTermFilteredPresearcher(4);
+  }
+
+  public void testSimpleBoolean() throws IOException {
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(
+          new MonitorQuery("1", parse("field:\"hello world\"")),
+          new MonitorQuery("2", parse("field:world")),
+          new MonitorQuery("3", parse("field:\"hello there world\"")),
+          new MonitorQuery("4", parse("field:\"this and that\"")));
+
+      MatchingQueries<QueryMatch> matches = monitor.match(buildDoc("field", "hello world and goodbye"),
+          QueryMatch.SIMPLE_MATCHER);
+      assertEquals(2, matches.getQueriesRun());
+      assertNotNull(matches.matches("1"));
+    }
+  }
+
+  public void testComplexBoolean() throws IOException {
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", parse("field:(+foo +bar +(badger cormorant))")));
+
+      MatchingQueries<QueryMatch> matches
+          = monitor.match(buildDoc("field", "a badger walked into a bar"), QueryMatch.SIMPLE_MATCHER);
+      assertEquals(0, matches.getMatchCount());
+      assertEquals(0, matches.getQueriesRun());
+
+      matches = monitor.match(buildDoc("field", "foo badger cormorant"), QueryMatch.SIMPLE_MATCHER);
+      assertEquals(0, matches.getMatchCount());
+      assertEquals(0, matches.getQueriesRun());
+
+      matches = monitor.match(buildDoc("field", "bar badger foo"), QueryMatch.SIMPLE_MATCHER);
+      assertEquals(1, matches.getMatchCount());
+    }
+
+  }
+
+  public void testQueryBuilder() throws IOException {
+
+    IndexWriterConfig iwc = new IndexWriterConfig(new KeywordAnalyzer());
+    Presearcher presearcher = createPresearcher();
+
+    Directory dir = new ByteBuffersDirectory();
+    IndexWriter writer = new IndexWriter(dir, iwc);
+    MonitorConfiguration config = new MonitorConfiguration(){
+      @Override
+      public IndexWriter buildIndexWriter() {
+        return writer;
+      }
+    };
+    try (Monitor monitor = new Monitor(ANALYZER, presearcher, config)) {
+
+      monitor.register(new MonitorQuery("1", parse("f:test")));
+
+      try (IndexReader reader = DirectoryReader.open(writer, false, false)) {
+
+        MemoryIndex mindex = new MemoryIndex();
+        mindex.addField("f", "this is a test document", WHITESPACE);
+        LeafReader docsReader = (LeafReader) mindex.createSearcher().getIndexReader();
+
+        QueryIndex.QueryTermFilter termFilter = new QueryIndex.QueryTermFilter(reader);
+
+        BooleanQuery q = (BooleanQuery) presearcher.buildQuery(docsReader, termFilter);
+        BooleanQuery expected = new BooleanQuery.Builder()
+            .add(should(new BooleanQuery.Builder()
+                .add(must(new BooleanQuery.Builder().add(should(new TermInSetQuery("f_0", new BytesRef("test")))).build()))
+                .add(must(new BooleanQuery.Builder().add(should(new TermInSetQuery("f_1", new BytesRef("test")))).build()))
+                .add(must(new BooleanQuery.Builder().add(should(new TermInSetQuery("f_2", new BytesRef("test")))).build()))
+                .add(must(new BooleanQuery.Builder().add(should(new TermInSetQuery("f_3", new BytesRef("test")))).build()))
+                .build()))
+            .add(should(new TermQuery(new Term("__anytokenfield", "__ANYTOKEN__"))))
+            .build();
+
+        assertEquals(expected, q);
+      }
+
+    }
+
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestParallelMatcher.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestParallelMatcher.java
new file mode 100644
index 0000000..2ede942
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestParallelMatcher.java
@@ -0,0 +1,28 @@
+/*
+ * 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.util.concurrent.ExecutorService;
+
+public class TestParallelMatcher extends ConcurrentMatcherTestBase {
+
+  @Override
+  protected <T extends QueryMatch> MatcherFactory<T> matcherFactory(ExecutorService executor, MatcherFactory<T> factory, int threads) {
+    return ParallelMatcher.factory(executor, factory, threads);
+  }
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestPartitionMatcher.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestPartitionMatcher.java
new file mode 100644
index 0000000..cc953da
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestPartitionMatcher.java
@@ -0,0 +1,60 @@
+/*
+ * 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.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+
+public class TestPartitionMatcher extends ConcurrentMatcherTestBase {
+
+  @Override
+  protected <T extends QueryMatch> MatcherFactory<T> matcherFactory(ExecutorService executor, MatcherFactory<T> factory, int threads) {
+    return PartitionMatcher.factory(executor, factory, threads);
+  }
+
+  public void testPartitions() {
+
+    List<String> terms = Arrays.asList("1", "2", "3", "4", "5", "6", "7", "8", "9", "10");
+
+    List<List<String>> partitions = PartitionMatcher.partition(terms, 2);
+    assertTrue(partitions.contains(Arrays.asList("1", "2", "3", "4", "5")));
+    assertTrue(partitions.contains(Arrays.asList("6", "7", "8", "9", "10")));
+
+    partitions = PartitionMatcher.partition(terms, 3);
+    assertTrue(partitions.contains(Arrays.asList("1", "2", "3")));
+    assertTrue(partitions.contains(Arrays.asList("4", "5", "6")));
+    assertTrue(partitions.contains(Arrays.asList("7", "8", "9", "10")));
+
+    partitions = PartitionMatcher.partition(terms, 4);
+    assertTrue(partitions.contains(Arrays.asList("1", "2")));
+    assertTrue(partitions.contains(Arrays.asList("3", "4", "5")));
+    assertTrue(partitions.contains(Arrays.asList("6", "7")));
+    assertTrue(partitions.contains(Arrays.asList("8", "9", "10")));
+
+    partitions = PartitionMatcher.partition(terms, 6);
+    assertTrue(partitions.contains(Collections.singletonList("1")));
+    assertTrue(partitions.contains(Arrays.asList("2", "3")));
+    assertTrue(partitions.contains(Arrays.asList("4", "5")));
+    assertTrue(partitions.contains(Collections.singletonList("6")));
+    assertTrue(partitions.contains(Arrays.asList("7", "8")));
+    assertTrue(partitions.contains(Arrays.asList("9", "10")));
+
+  }
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestPresearcherMatchCollector.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestPresearcherMatchCollector.java
new file mode 100644
index 0000000..f5365c1
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestPresearcherMatchCollector.java
@@ -0,0 +1,61 @@
+/*
+ * 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.document.Document;
+import org.apache.lucene.document.Field;
+
+import static org.hamcrest.CoreMatchers.containsString;
+
+public class TestPresearcherMatchCollector extends MonitorTestBase {
+
+  public void testMatchCollectorShowMatches() throws IOException {
+
+    try (Monitor monitor = new Monitor(ANALYZER, new TermFilteredPresearcher())) {
+      monitor.register(new MonitorQuery("1", parse("test")));
+      monitor.register(new MonitorQuery("2", parse("foo bar -baz f2:quuz")));
+      monitor.register(new MonitorQuery("3", parse("foo -test")));
+      monitor.register(new MonitorQuery("4", parse("baz")));
+      assertEquals(4, monitor.getQueryCount());
+
+      Document doc = new Document();
+      doc.add(newTextField(FIELD, "this is a foo test", Field.Store.NO));
+      doc.add(newTextField("f2", "quuz", Field.Store.NO));
+
+      PresearcherMatches<QueryMatch> matches = monitor.debug(doc, QueryMatch.SIMPLE_MATCHER);
+
+      assertNotNull(matches.match("1", 0));
+      assertEquals(" field:test", matches.match("1", 0).presearcherMatches);
+      assertNotNull(matches.match("1", 0).queryMatch);
+
+      assertNotNull(matches.match("2", 0));
+      String pm = matches.match("2", 0).presearcherMatches;
+      assertThat(pm, containsString("field:foo"));
+      assertThat(pm, containsString("f2:quuz"));
+
+      assertNotNull(matches.match("3", 0));
+      assertEquals(" field:foo", matches.match("3", 0).presearcherMatches);
+      assertNull(matches.match("3", 0).queryMatch);
+
+      assertNull(matches.match("4", 0));
+    }
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestQueryAnalyzer.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestQueryAnalyzer.java
new file mode 100644
index 0000000..f74361a
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestQueryAnalyzer.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.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestQueryAnalyzer extends LuceneTestCase {
+
+  public static final QueryAnalyzer analyzer = new QueryAnalyzer();
+
+  private Set<Term> collectTerms(QueryTree tree) {
+    Set<Term> terms = new HashSet<>();
+    tree.collectTerms((f, b) -> terms.add(new Term(f, b)));
+    return terms;
+  }
+
+  private Set<Term> collectTerms(Query query) {
+    return collectTerms(analyzer.buildTree(query, TermWeightor.DEFAULT));
+  }
+
+  public void testAdvancesCollectDifferentTerms() {
+
+    Query q = MonitorTestBase.parse("field:(+hello +goodbye)");
+    QueryTree querytree = analyzer.buildTree(q, TermWeightor.DEFAULT);
+
+    Set<Term> expected = Collections.singleton(new Term("field", "goodbye"));
+    assertEquals(expected, collectTerms(querytree));
+
+    assertTrue(querytree.advancePhase(0));
+    expected = Collections.singleton(new Term("field", "hello"));
+    assertEquals(expected, collectTerms(querytree));
+
+    assertFalse(querytree.advancePhase(0));
+
+    assertEquals(expected, collectTerms(querytree));
+
+  }
+
+  public void testDisjunctionsWithAnyClausesOnlyReturnANYTOKEN() {
+
+    // disjunction containing a pure negative - we can't narrow this down
+    Query q = MonitorTestBase.parse("hello goodbye (*:* -term)");
+
+    Set<Term> terms = collectTerms(q);
+    assertEquals(1, terms.size());
+    assertEquals(TermFilteredPresearcher.ANYTOKEN_FIELD, terms.iterator().next().field());
+
+  }
+
+  public void testConjunctionsDoNotAdvanceOverANYTOKENs() {
+
+    Query q = MonitorTestBase.parse("+hello +howdyedo +(goodbye (*:* -whatever))");
+    QueryTree tree = analyzer.buildTree(q, TermWeightor.DEFAULT);
+
+    Set<Term> expected = Collections.singleton(new Term("field", "howdyedo"));
+    assertEquals(expected, collectTerms(tree));
+
+    assertTrue(tree.advancePhase(0));
+    expected = Collections.singleton(new Term("field", "hello"));
+    assertEquals(expected, collectTerms(tree));
+
+    assertFalse(tree.advancePhase(0));
+    assertEquals(expected, collectTerms(tree));
+
+  }
+
+  public void testConjunctionsCannotAdvanceOverMinWeightedTokens() {
+
+    TermWeightor weightor = TermWeightor.combine(
+        TermWeightor.termWeightor(0.1, new BytesRef("startterm")),
+        TermWeightor.lengthWeightor(1, 1));
+
+    QueryAnalyzer analyzer = new QueryAnalyzer();
+
+    Query q = MonitorTestBase.parse("+startterm +hello +goodbye");
+    QueryTree tree = analyzer.buildTree(q, weightor);
+
+    Set<Term> expected = Collections.singleton(new Term("field", "goodbye"));
+    assertEquals(expected, collectTerms(tree));
+
+    assertTrue(tree.advancePhase(0.5));
+    expected = Collections.singleton(new Term("field", "hello"));
+    assertEquals(expected, collectTerms(tree));
+
+    assertFalse(tree.advancePhase(0.5));
+
+  }
+
+  public void testNestedConjunctions() {
+
+    Query q = MonitorTestBase.parse("+(+(+(+aaaa +cc) +(+d +bbb)))");
+    QueryTree tree = analyzer.buildTree(q, TermWeightor.DEFAULT);
+
+    Set<Term> expected = Collections.singleton(new Term("field", "aaaa"));
+    assertEquals(expected, collectTerms(tree));
+    assertTrue(tree.advancePhase(0));
+
+    expected = Collections.singleton(new Term("field", "bbb"));
+    assertEquals(expected, collectTerms(tree));
+    assertTrue(tree.advancePhase(0));
+
+    expected = Collections.singleton(new Term("field", "cc"));
+    assertEquals(expected, collectTerms(tree));
+    assertTrue(tree.advancePhase(0));
+
+    expected = Collections.singleton(new Term("field", "d"));
+    assertEquals(expected, collectTerms(tree));
+    assertFalse(tree.advancePhase(0));
+
+  }
+
+  public void testNestedDisjunctions() {
+
+    Query q = MonitorTestBase.parse("+(+((+aaaa +cc) (+dd +bbb +f)))");
+    QueryTree tree = analyzer.buildTree(q, TermWeightor.DEFAULT);
+
+    Set<Term> expected = new HashSet<>(Arrays.asList(
+        new Term("field", "aaaa"),
+        new Term("field", "bbb"
+    )));
+    assertEquals(expected, collectTerms(tree));
+    assertTrue(tree.advancePhase(0));
+
+    expected = new HashSet<>(Arrays.asList(
+        new Term("field", "cc"),
+        new Term("field", "dd")
+    ));
+    assertEquals(expected, collectTerms(tree));
+    assertTrue(tree.advancePhase(0));
+
+    expected = new HashSet<>(Arrays.asList(
+        new Term("field", "cc"),
+        new Term("field", "f")
+    ));
+    assertEquals(expected, collectTerms(tree));
+    assertFalse(tree.advancePhase(0));
+  }
+
+  public void testMinWeightAdvances() {
+    QueryTree tree = QueryTree.disjunction(
+        QueryTree.conjunction(
+            QueryTree.term(new Term("field", "term1"), 1),
+            QueryTree.term(new Term("field", "term2"), 0.1),
+            QueryTree.anyTerm("*:*")
+        ),
+        QueryTree.conjunction(
+            QueryTree.disjunction(
+                QueryTree.term(new Term("field", "term4"), 0.2),
+                QueryTree.term(new Term("field", "term5"), 1)
+            ),
+            QueryTree.term(new Term("field", "term3"), 0.5)
+        )
+    );
+
+    Set<Term> expected = new HashSet<>(Arrays.asList(
+        new Term("field", "term1"),
+        new Term("field", "term3")
+    ));
+    assertEquals(expected, collectTerms(tree));
+    assertTrue(tree.advancePhase(0.1f));
+
+    expected = new HashSet<>(Arrays.asList(
+        new Term("field", "term1"),
+        new Term("field", "term4"),
+        new Term("field", "term5")
+    ));
+    assertEquals(expected, collectTerms(tree));
+    assertFalse(tree.advancePhase(0.1f));
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestQueryDecomposer.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestQueryDecomposer.java
new file mode 100644
index 0000000..1de4d86
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestQueryDecomposer.java
@@ -0,0 +1,110 @@
+/*
+ * 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.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.DisjunctionMaxQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+
+public class TestQueryDecomposer extends MonitorTestBase  {
+
+  private static final QueryDecomposer decomposer = new QueryDecomposer();
+
+  public void testConjunctionsAreNotDecomposed() {
+    Query q = parse("+hello world");
+    Set<Query> expected = Collections.singleton(parse("+hello world"));
+    assertEquals(expected, decomposer.decompose(q));
+  }
+
+  public void testSimpleDisjunctions() {
+    Query q = parse("hello world");
+    Set<Query> expected = new HashSet<>(Arrays.asList(parse("hello"), parse("world")));
+    assertEquals(expected, decomposer.decompose(q));
+  }
+
+  public void testNestedDisjunctions() {
+    Query q = parse("(hello goodbye) world");
+    Set<Query> expected = new HashSet<>(Arrays.asList(parse("hello"), parse("goodbye"), parse("world")));
+    assertEquals(expected, decomposer.decompose(q));
+  }
+
+  public void testExclusions() {
+    Set<Query> expected = new HashSet<>(Arrays.asList(parse("+hello -goodbye"), parse("+world -goodbye")));
+    assertEquals(expected, decomposer.decompose(parse("hello world -goodbye")));
+  }
+
+  public void testNestedExclusions() {
+    Set<Query> expected
+        = new HashSet<>(Arrays.asList(parse("+(+hello -goodbye) -greeting"), parse("+(+world -goodbye) -greeting")));
+    assertEquals(expected, decomposer.decompose(parse("((hello world) -goodbye) -greeting")));
+  }
+
+  public void testSingleValuedConjunctions() {
+    Set<Query> expected = new HashSet<>(Arrays.asList(parse("hello"), parse("world")));
+    assertEquals(expected, decomposer.decompose(parse("+(hello world)")));
+  }
+
+  public void testSingleValuedConjunctWithExclusions() {
+    Set<Query> expected = new HashSet<>(Arrays.asList(parse("+hello -goodbye"), parse("+world -goodbye")));
+    assertEquals(expected, decomposer.decompose(parse("+(hello world) -goodbye")));
+  }
+
+  public void testBoostsArePreserved() {
+    Set<Query> expected = new HashSet<>(Arrays.asList(parse("hello^0.7"), parse("world^0.7")));
+    assertEquals(expected, decomposer.decompose(parse("+(hello world)^0.7")));
+    expected = new HashSet<>(Arrays.asList(parse("+hello^0.7 -goodbye"), parse("+world^0.7 -goodbye")));
+    assertEquals(expected, decomposer.decompose(parse("+(hello world)^0.7 -goodbye")));
+    expected = new HashSet<>(Arrays.asList(parse("(hello^0.5)^0.8"), parse("world^0.8")));
+    assertEquals(expected, decomposer.decompose(parse("+(hello^0.5 world)^0.8")));
+  }
+
+  public void testDisjunctionMaxDecomposition() {
+    Query q = new DisjunctionMaxQuery(
+        Arrays.asList(new TermQuery(new Term("f", "t1")), new TermQuery(new Term("f", "t2"))), 0.1f
+    );
+    Set<Query> expected = new HashSet<>(Arrays.asList(parse("f:t1"), parse("f:t2")));
+    assertEquals(expected, decomposer.decompose(q));
+  }
+
+  public void testNestedDisjunctionMaxDecomposition() {
+    Query q = new DisjunctionMaxQuery(
+        Arrays.asList(parse("hello goodbye"), parse("world")), 0.1f
+    );
+    Set<Query> expected = new HashSet<>(Arrays.asList(parse("hello"), parse("goodbye"), parse("world")));
+    assertEquals(expected, decomposer.decompose(q));
+  }
+
+  public void testFilterAndShouldClause() {
+    final Query shouldTermQuery = new TermQuery(new Term("f", "should"));
+    final Query filterTermQuery = new TermQuery(new Term("f", "filter"));
+    Query q = new BooleanQuery.Builder()
+        .add(shouldTermQuery, BooleanClause.Occur.SHOULD)
+        .add(filterTermQuery, BooleanClause.Occur.FILTER)
+        .build();
+
+    assertEquals(Collections.singleton(q), decomposer.decompose(q));
+  }
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestQueryTermComparators.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestQueryTermComparators.java
new file mode 100644
index 0000000..c2d0c5e
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestQueryTermComparators.java
@@ -0,0 +1,103 @@
+/*
+ * 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.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestQueryTermComparators extends LuceneTestCase {
+
+  public void testAnyTokensAreNotPreferred() {
+
+    QueryTree node1 = QueryTree.term("f", new BytesRef("foo"), 1.0);
+    QueryTree node2 = QueryTree.anyTerm("*:*");
+
+    QueryTree conjunction = QueryTree.conjunction(node1, node2);
+    Set<Term> terms = new HashSet<>();
+    conjunction.collectTerms((f, b) -> terms.add(new Term(f, b)));
+    Set<Term> expected = Collections.singleton(new Term("f", "foo"));
+    assertEquals(expected, terms);
+  }
+
+  public void testHigherWeightsArePreferred() {
+
+    QueryTree node1 = QueryTree.term(new Term("f", "foo"), 1);
+    QueryTree node2 = QueryTree.term(new Term("f", "foobar"), 1.5);
+
+    QueryTree conjunction = QueryTree.conjunction(node1, node2);
+    Set<Term> terms = new HashSet<>();
+    conjunction.collectTerms((f, b) -> terms.add(new Term(f, b)));
+    Set<Term> expected = Collections.singleton(new Term("f", "foobar"));
+    assertEquals(expected, terms);
+  }
+
+  public void testShorterTermListsArePreferred() {
+
+    Term term = new Term("f", "foobar");
+
+    QueryTree node1 = QueryTree.term(term, 1);
+    QueryTree node2 = QueryTree.disjunction(
+        QueryTree.term(term, 1),
+        QueryTree.term(term, 1));
+
+    QueryTree conjunction = QueryTree.conjunction(node1, node2);
+    Set<Term> terms = new HashSet<>();
+    conjunction.collectTerms((f, b) -> terms.add(new Term(f, b)));
+    assertEquals(1, terms.size());
+
+  }
+
+  public void testFieldWeights() {
+    TermWeightor weightor = TermWeightor.fieldWeightor(1.5, "g");
+    assertEquals(1, weightor.applyAsDouble(new Term("f", "foo")), 0);
+    assertEquals(1.5f, weightor.applyAsDouble(new Term("g", "foo")), 0);
+  }
+
+  public void testTermWeights() {
+    TermWeightor weight = TermWeightor.termWeightor(0.01f, new BytesRef("START"));
+    assertEquals(0.01f, weight.applyAsDouble(new Term("f", "START")), 0);
+  }
+
+  public void testTermFrequencyNorms() {
+
+    Map<String, Integer> termfreqs = new HashMap<>();
+    termfreqs.put("france", 31635);
+    termfreqs.put("s", 47088);
+    TermWeightor weight = TermWeightor.termFreqWeightor(termfreqs, 100, 0.8);
+
+    assertTrue(weight.applyAsDouble(new Term("f", "france")) >
+        weight.applyAsDouble(new Term("f", "s")));
+
+  }
+
+  public void testFieldSpecificTermWeightNorms() {
+    TermWeightor weight = TermWeightor.termAndFieldWeightor(0.1,
+        new Term("field1", "f"),
+        new Term("field1", "g"));
+    assertEquals(0.1, weight.applyAsDouble(new Term("field1", "f")), 0);
+    assertEquals(1, weight.applyAsDouble(new Term("field2", "f")), 0);
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestQueryTermFilter.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestQueryTermFilter.java
new file mode 100644
index 0000000..9a5b966
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestQueryTermFilter.java
@@ -0,0 +1,52 @@
+/*
+ * 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.Collections;
+import java.util.function.BiPredicate;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestQueryTermFilter extends LuceneTestCase {
+
+  private static final String FIELD = "f";
+
+  public void testFiltersAreRemoved() throws IOException {
+
+    try (QueryIndex qi = new QueryIndex(new MonitorConfiguration(), new TermFilteredPresearcher())) {
+      qi.commit(Collections.singletonList(new MonitorQuery("1", new TermQuery(new Term(FIELD, "term")))));
+      assertEquals(1, qi.termFilters.size());
+      BiPredicate<String, BytesRef> filter = qi.termFilters.values().iterator().next();
+      assertTrue(filter.test(FIELD, new BytesRef("term")));
+      assertFalse(filter.test(FIELD, new BytesRef("term2")));
+
+      qi.commit(Collections.singletonList(new MonitorQuery("2", new TermQuery(new Term(FIELD, "term2")))));
+      assertEquals(1, qi.termFilters.size());
+
+      filter = qi.termFilters.values().iterator().next();
+      assertTrue(filter.test(FIELD, new BytesRef("term")));
+      assertTrue(filter.test(FIELD, new BytesRef("term2")));
+      assertFalse(filter.test(FIELD, new BytesRef("term3")));
+    }
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestRegexpQueryHandler.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestRegexpQueryHandler.java
new file mode 100644
index 0000000..8836b88
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestRegexpQueryHandler.java
@@ -0,0 +1,84 @@
+/*
+ * 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.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.RegexpQuery;
+
+public class TestRegexpQueryHandler extends BaseTokenStreamTestCase {
+
+  public void testTermStreamWrapping() throws IOException {
+
+    CustomQueryHandler handler
+        = new RegexpQueryHandler("FOO", 10, "__wibble__", Collections.singleton("field1"));
+
+    try (Analyzer input = new WhitespaceAnalyzer()) {
+
+      // field1 is in the excluded set, so nothing should happen
+      assertTokenStreamContents(handler.wrapTermStream("field1", input.tokenStream("field1", "hello world")),
+          new String[]{ "hello", "world" });
+
+      // field2 is not excluded
+      assertTokenStreamContents(handler.wrapTermStream("field2", input.tokenStream("field2", "harm alarm asdasasdasdasd")),
+          new String[]{
+              "harm", "harmFOO", "harFOO", "haFOO", "hFOO", "armFOO", "arFOO", "aFOO", "rmFOO", "rFOO", "mFOO", "FOO",
+              "alarm", "alarmFOO", "alarFOO", "alaFOO", "alFOO", "larmFOO", "larFOO", "laFOO", "lFOO",
+              "asdasasdasdasd", "__wibble__"
+          });
+    }
+  }
+
+  private Set<Term> collectTerms(Query q) {
+    QueryAnalyzer builder = new QueryAnalyzer(Collections.singletonList(
+        new RegexpQueryHandler("XX", 30, "WILDCARD", null)));
+    QueryTree tree = builder.buildTree(q, TermWeightor.DEFAULT);
+    Set<Term> terms = new HashSet<>();
+    tree.collectTerms((f, b) -> terms.add(new Term(f, b)));
+    return terms;
+  }
+
+  public void testRegexpExtractor() {
+
+    Set<Term> expected = new HashSet<>(Arrays.asList(
+        new Term("field", "califragilisticXX"),
+        new Term("field", "WILDCARD")));
+    assertEquals(expected, collectTerms(new RegexpQuery(new Term("field", "super.*califragilistic"))));
+
+    expected = new HashSet<>(Arrays.asList(
+        new Term("field", "hellXX"),
+        new Term("field", "WILDCARD")));
+    assertEquals(expected, collectTerms(new RegexpQuery(new Term("field", "hell."))));
+
+    expected = new HashSet<>(Arrays.asList(
+        new Term("field", "heXX"),
+        new Term("field", "WILDCARD")));
+    assertEquals(expected, collectTerms(new RegexpQuery(new Term("field", "hel?o"))));
+
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestSimilarities.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestSimilarities.java
new file mode 100644
index 0000000..6420415
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestSimilarities.java
@@ -0,0 +1,50 @@
+/*
+ * 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 org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.search.similarities.ClassicSimilarity;
+import org.apache.lucene.search.similarities.Similarity;
+
+public class TestSimilarities extends MonitorTestBase {
+
+  public void testNonStandardSimilarity() throws Exception {
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", MonitorTestBase.parse("test")));
+
+      Similarity similarity = new ClassicSimilarity() {
+        @Override
+        public float tf(float freq) {
+          return 1000f;
+        }
+      };
+
+      Document doc = new Document();
+      doc.add(newTextField("field", "this is a test", Field.Store.NO));
+
+      MatchingQueries<ScoringMatch> standard = monitor.match(doc, ScoringMatch.matchWithSimilarity(new ClassicSimilarity()));
+      MatchingQueries<ScoringMatch> withSim = monitor.match(doc, ScoringMatch.matchWithSimilarity(similarity));
+
+      float standScore = standard.getMatches().iterator().next().getScore();
+      float simScore = withSim.getMatches().iterator().next().getScore();
+      assertEquals(standScore, simScore / 1000, 0.1f);
+    }
+  }
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestSimpleMatcher.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestSimpleMatcher.java
new file mode 100644
index 0000000..5f37116
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestSimpleMatcher.java
@@ -0,0 +1,40 @@
+/*
+ * 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.document.Document;
+import org.apache.lucene.document.Field;
+
+public class TestSimpleMatcher extends MonitorTestBase {
+
+  public void testSimpleMatcher() throws IOException {
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(
+          new MonitorQuery("1", parse("test")),
+          new MonitorQuery("2", parse("wibble")));
+      Document doc = new Document();
+      doc.add(newTextField(FIELD, "test", Field.Store.NO));
+
+      MatchingQueries<QueryMatch> matches = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertNotNull(matches.matches("1"));
+    }
+  }
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestSpanExtractors.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestSpanExtractors.java
new file mode 100644
index 0000000..7f4e36b
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestSpanExtractors.java
@@ -0,0 +1,134 @@
+/*
+ * 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.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.RegexpQuery;
+import org.apache.lucene.search.spans.FieldMaskingSpanQuery;
+import org.apache.lucene.search.spans.SpanBoostQuery;
+import org.apache.lucene.search.spans.SpanContainingQuery;
+import org.apache.lucene.search.spans.SpanFirstQuery;
+import org.apache.lucene.search.spans.SpanMultiTermQueryWrapper;
+import org.apache.lucene.search.spans.SpanNearQuery;
+import org.apache.lucene.search.spans.SpanOrQuery;
+import org.apache.lucene.search.spans.SpanQuery;
+import org.apache.lucene.search.spans.SpanTermQuery;
+import org.apache.lucene.search.spans.SpanWithinQuery;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestSpanExtractors extends LuceneTestCase  {
+
+  private static final QueryAnalyzer treeBuilder = new QueryAnalyzer();
+
+  private Set<Term> collectTerms(Query q) {
+    QueryTree tree = treeBuilder.buildTree(q, TermWeightor.DEFAULT);
+    Set<Term> terms = new HashSet<>();
+    tree.collectTerms((f, b) -> terms.add(new Term(f, b)));
+    return terms;
+  }
+
+  public void testOrderedNearExtractor() {
+    SpanNearQuery q = new SpanNearQuery(new SpanQuery[]{
+        new SpanTermQuery(new Term("field1", "term1")),
+        new SpanTermQuery(new Term("field1", "term"))
+    }, 0, true);
+
+    Set<Term> expected = Collections.singleton(new Term("field1", "term1"));
+    assertEquals(expected, collectTerms(q));
+  }
+
+  public void testOrderedNearWithWildcardExtractor() {
+    SpanNearQuery q = new SpanNearQuery(new SpanQuery[]{
+        new SpanMultiTermQueryWrapper<>(new RegexpQuery(new Term("field", "super.*cali.*"))),
+        new SpanTermQuery(new Term("field", "is"))
+    }, 0, true);
+
+    Set<Term> expected = Collections.singleton(new Term("field", "is"));
+    assertEquals(expected, collectTerms(q));
+  }
+
+  public void testSpanOrExtractor() {
+    SpanOrQuery or = new SpanOrQuery(new SpanTermQuery(new Term("field", "term1")),
+        new SpanTermQuery(new Term("field", "term2")));
+    Set<Term> expected = new HashSet<>(Arrays.asList(
+        new Term("field", "term1"),
+        new Term("field", "term2")
+    ));
+    assertEquals(expected, collectTerms(or));
+  }
+
+  public void testSpanMultiTerms() {
+    SpanQuery q = new SpanMultiTermQueryWrapper<>(new RegexpQuery(new Term("field", "term.*")));
+    Set<Term> terms = collectTerms(q);
+    assertEquals(1, terms.size());
+    assertEquals(TermFilteredPresearcher.ANYTOKEN_FIELD, terms.iterator().next().field());
+  }
+
+  public void testSpanWithin() {
+    Term t1 = new Term("field", "term1");
+    Term t2 = new Term("field", "term22");
+    Term t3 = new Term("field", "term333");
+    SpanWithinQuery swq = new SpanWithinQuery(
+        SpanNearQuery.newOrderedNearQuery("field")
+            .addClause(new SpanTermQuery(t1))
+            .addClause(new SpanTermQuery(t2))
+            .build(),
+        new SpanTermQuery(t3));
+
+    assertEquals(Collections.singleton(t3), collectTerms(swq));
+  }
+
+  public void testSpanContains() {
+    Term t1 = new Term("field", "term1");
+    Term t2 = new Term("field", "term22");
+    Term t3 = new Term("field", "term333");
+    SpanContainingQuery swq = new SpanContainingQuery(
+        SpanNearQuery.newOrderedNearQuery("field")
+            .addClause(new SpanTermQuery(t1))
+            .addClause(new SpanTermQuery(t2))
+            .build(),
+        new SpanTermQuery(t3));
+
+    assertEquals(Collections.singleton(t3), collectTerms(swq));
+  }
+
+  public void testSpanBoost() {
+    Term t1 = new Term("field", "term1");
+    SpanBoostQuery q = new SpanBoostQuery(new SpanTermQuery(t1), 0.1f);
+    assertEquals(Collections.singleton(t1), collectTerms(q));
+  }
+
+  public void testFieldMaskingSpanQuery() {
+    Term t1 = new Term("field", "term1");
+    FieldMaskingSpanQuery q = new FieldMaskingSpanQuery(new SpanTermQuery(t1), "field2");
+    assertEquals(Collections.singleton(t1), collectTerms(q));
+  }
+
+  public void testSpanPositionQuery() {
+    Term t1 = new Term("field", "term");
+    Query q = new SpanFirstQuery(new SpanTermQuery(t1), 10);
+    assertEquals(Collections.singleton(t1), collectTerms(q));
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestSuffixingNGramTokenizer.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestSuffixingNGramTokenizer.java
new file mode 100644
index 0000000..5bb2e75
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestSuffixingNGramTokenizer.java
@@ -0,0 +1,67 @@
+/*
+ * 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.analysis.Analyzer;
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.core.WhitespaceTokenizer;
+
+public class TestSuffixingNGramTokenizer extends BaseTokenStreamTestCase {
+
+  private Analyzer analyzer = new Analyzer() {
+    @Override
+    protected TokenStreamComponents createComponents(String fieldName) {
+      Tokenizer source = new WhitespaceTokenizer();
+      TokenStream sink = new SuffixingNGramTokenFilter(source, "XX", "ANY", 10);
+      return new TokenStreamComponents(source, sink);
+    }
+  };
+
+  public void testTokensAreSuffixed() throws IOException {
+    assertAnalyzesTo(analyzer, "term", new String[]{
+        "term", "termXX", "terXX", "teXX", "tXX", "ermXX", "erXX", "eXX", "rmXX", "rXX", "mXX", "XX"
+    });
+  }
+
+  public void testRepeatedSuffixesAreNotEmitted() throws IOException {
+    assertAnalyzesTo(analyzer, "arm harm term", new String[]{
+        "arm", "armXX", "arXX", "aXX", "rmXX", "rXX", "mXX", "XX",
+        "harm", "harmXX", "harXX", "haXX", "hXX",
+        "term", "termXX", "terXX", "teXX", "tXX", "ermXX", "erXX", "eXX"
+    });
+  }
+
+  public void testRepeatedInfixesAreNotEmitted() throws IOException {
+    assertAnalyzesTo(analyzer, "alarm alas harm", new String[]{
+        "alarm", "alarmXX", "alarXX", "alaXX", "alXX", "aXX",
+        "larmXX", "larXX", "laXX", "lXX", "armXX", "arXX", "rmXX", "rXX", "mXX", "XX",
+        "alas", "alasXX", "lasXX", "asXX", "sXX", "harm", "harmXX", "harXX", "haXX", "hXX"
+    });
+  }
+
+  public void testLengthyTokensAreNotNgrammed() throws IOException {
+    assertAnalyzesTo(analyzer, "alongtermthatshouldntbengrammed", new String[]{
+        "alongtermthatshouldntbengrammed", "ANY"
+    });
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestTermPresearcher.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestTermPresearcher.java
new file mode 100644
index 0000000..64a3069
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestTermPresearcher.java
@@ -0,0 +1,152 @@
+/*
+ * 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.Map;
+
+import org.apache.lucene.analysis.core.KeywordAnalyzer;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.memory.MemoryIndex;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.TermInSetQuery;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.ByteBuffersDirectory;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefHash;
+
+public class TestTermPresearcher extends PresearcherTestBase {
+
+  public void testFiltersOnTermQueries() throws IOException {
+
+    MonitorQuery query1
+        = new MonitorQuery("1", parse("furble"));
+    MonitorQuery query2
+        = new MonitorQuery("2", parse("document"));
+    MonitorQuery query3 = new MonitorQuery("3", parse("\"a document\""));  // will be selected but not match
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(query1, query2, query3);
+
+      Map<String, Long> timings = new HashMap<>();
+      QueryTimeListener timeListener =
+          (queryId, timeInNanos) -> timings.compute(queryId, (q, t) -> t == null ? timeInNanos : t + timeInNanos);
+      MatchingQueries<QueryMatch> matches = monitor.match(buildDoc(TEXTFIELD, "this is a test document"),
+          QueryTimeListener.timingMatcher(QueryMatch.SIMPLE_MATCHER, timeListener));
+      assertEquals(1, matches.getMatchCount());
+      assertNotNull(matches.matches("2"));
+      assertEquals(2, matches.getQueriesRun());
+      assertEquals(2, timings.size());
+      assertTrue(timings.keySet().contains("2"));
+      assertTrue(timings.keySet().contains("3"));
+    }
+  }
+
+  public void testIgnoresTermsOnNotQueries() throws IOException {
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", parse("document -test")));
+
+      MatchingQueries<QueryMatch> matches = monitor.match(buildDoc(TEXTFIELD, "this is a test document"), QueryMatch.SIMPLE_MATCHER);
+      assertEquals(0, matches.getMatchCount());
+      assertEquals(1, matches.getQueriesRun());
+
+      matches = monitor.match(buildDoc(TEXTFIELD, "weeble sclup test"), QueryMatch.SIMPLE_MATCHER);
+      assertEquals(0, matches.getMatchCount());
+      assertEquals(0, matches.getQueriesRun());
+    }
+
+  }
+
+  public void testMatchesAnyQueries() throws IOException {
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", parse("/hell./")));
+
+      MatchingQueries<QueryMatch> matches = monitor.match(buildDoc(TEXTFIELD, "hello"), QueryMatch.SIMPLE_MATCHER);
+      assertEquals(1, matches.getMatchCount());
+      assertEquals(1, matches.getQueriesRun());
+    }
+
+  }
+
+  @Override
+  protected Presearcher createPresearcher() {
+    return new TermFilteredPresearcher();
+  }
+
+  public void testAnyTermsAreCorrectlyAnalyzed() {
+
+    QueryAnalyzer analyzer = new QueryAnalyzer();
+    QueryTree qt = analyzer.buildTree(new MatchAllDocsQuery(), TermFilteredPresearcher.DEFAULT_WEIGHTOR);
+
+    TermFilteredPresearcher presearcher = new TermFilteredPresearcher();
+    Map<String, BytesRefHash> extractedTerms = presearcher.collectTerms(qt);
+    assertEquals(1, extractedTerms.size());
+
+  }
+
+  public void testQueryBuilder() throws IOException {
+
+    Presearcher presearcher = createPresearcher();
+
+    IndexWriterConfig iwc = new IndexWriterConfig(new KeywordAnalyzer());
+    Directory dir = new ByteBuffersDirectory();
+    IndexWriter writer = new IndexWriter(dir, iwc);
+    MonitorConfiguration config = new MonitorConfiguration(){
+      @Override
+      public IndexWriter buildIndexWriter() {
+        return writer;
+      }
+    };
+
+    try (Monitor monitor = new Monitor(ANALYZER, presearcher, config)) {
+
+      monitor.register(new MonitorQuery("1", parse("f:test")));
+
+      try (IndexReader reader = DirectoryReader.open(writer, false, false)) {
+
+        MemoryIndex mindex = new MemoryIndex();
+        mindex.addField("f", "this is a test document", WHITESPACE);
+        LeafReader docsReader = (LeafReader) mindex.createSearcher().getIndexReader();
+
+        QueryIndex.QueryTermFilter termFilter = new QueryIndex.QueryTermFilter(reader);
+
+        BooleanQuery q = (BooleanQuery) presearcher.buildQuery(docsReader, termFilter);
+        BooleanQuery expected = new BooleanQuery.Builder()
+            .add(should(new BooleanQuery.Builder()
+                .add(should(new TermInSetQuery("f", new BytesRef("test")))).build()))
+            .add(should(new TermQuery(new Term("__anytokenfield", "__ANYTOKEN__"))))
+            .build();
+
+        assertEquals(expected, q);
+
+      }
+
+    }
+
+  }
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestTermsEnumTokenFilter.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestTermsEnumTokenFilter.java
new file mode 100644
index 0000000..93ff578
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestTermsEnumTokenFilter.java
@@ -0,0 +1,73 @@
+/*
+ * 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.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefIterator;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestTermsEnumTokenFilter extends LuceneTestCase {
+
+  final class LeapfrogTokenFilter extends TokenFilter {
+
+    final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
+
+    LeapfrogTokenFilter(TokenStream input) {
+      super(input);
+    }
+
+    @Override
+    public boolean incrementToken() throws IOException {
+      posIncAtt.setPositionIncrement(100000000);
+      return input.incrementToken();
+    }
+  }
+
+  public void testPosIncAttributeOverflow() throws IOException {
+
+    final BytesRef foo = new BytesRef("foo");
+    final BytesRef bar = new BytesRef("bar");
+
+    BytesRefIterator terms = new BytesRefIterator() {
+
+      long count = 1000;
+
+      @Override
+      public BytesRef next() throws IOException {
+        if (count-- > 100)
+          return foo;
+        if (count-- > 0)
+          return bar;
+        return null;
+      }
+    };
+
+    try (TokenStream ts = new LeapfrogTokenFilter(new TermsEnumTokenStream(terms))) {
+      while (ts.incrementToken()) {
+        // This tight loop will throw an exception if clearAttributes() is not called
+        // by TermsEnumTokenStream.  See issue #46
+      }
+    }
+  }
+
+}
diff --git a/lucene/monitor/src/test/org/apache/lucene/monitor/TestWildcardTermPresearcher.java b/lucene/monitor/src/test/org/apache/lucene/monitor/TestWildcardTermPresearcher.java
new file mode 100644
index 0000000..57fcd00
--- /dev/null
+++ b/lucene/monitor/src/test/org/apache/lucene/monitor/TestWildcardTermPresearcher.java
@@ -0,0 +1,80 @@
+/*
+ * 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.Collections;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+
+public class TestWildcardTermPresearcher extends PresearcherTestBase {
+
+  public void testFiltersWildcards() throws IOException {
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", parse("/hell.*/")));
+      assertEquals(1,
+          monitor.match(buildDoc(TEXTFIELD, "well hello there"), QueryMatch.SIMPLE_MATCHER).getMatchCount());
+      assertEquals(0, monitor.match(buildDoc(TEXTFIELD, "hi there"), QueryMatch.SIMPLE_MATCHER).getQueriesRun());
+    }
+  }
+
+  public void testNgramsOnlyMatchWildcards() throws IOException {
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", parse("hello")));
+      assertEquals(0, monitor.match(buildDoc(TEXTFIELD, "hellopolis"), QueryMatch.SIMPLE_MATCHER).getQueriesRun());
+    }
+  }
+
+  private static String repeat(String input, int size) {
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < size; i++) {
+      sb.append(input);
+    }
+    return sb.toString();
+  }
+
+  public void testLongTermsStillMatchWildcards() throws IOException {
+
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", parse("/a.*/")));
+
+      Document doc = new Document();
+      doc.add(newTextField(TEXTFIELD, repeat("a", RegexpQueryHandler.DEFAULT_MAX_TOKEN_SIZE + 1), Field.Store.NO));
+
+      MatchingQueries<QueryMatch> matches = monitor.match(doc, QueryMatch.SIMPLE_MATCHER);
+      assertEquals(1, matches.getQueriesRun());
+      assertNotNull(matches.matches("1"));
+    }
+
+  }
+
+  public void testCaseSensitivity() throws IOException {
+    try (Monitor monitor = newMonitor()) {
+      monitor.register(new MonitorQuery("1", parse("foo")));
+      assertEquals(1,
+          monitor.match(buildDoc(TEXTFIELD, "Foo foo"), QueryMatch.SIMPLE_MATCHER).getMatchCount());
+    }
+  }
+
+  @Override
+  protected Presearcher createPresearcher() {
+    return new TermFilteredPresearcher(TermWeightor.DEFAULT, Collections.singletonList(new RegexpQueryHandler()), Collections.emptySet());
+  }
+
+}