You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ju...@apache.org on 2021/06/09 20:42:00 UTC

[lucene-solr] branch exp updated (9bc5823 -> 8752057)

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

julietibs pushed a change to branch exp
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git.


 discard 9bc5823  LUCENE-9965: Add tooling to introspect query execution time (#144)
     new 8752057  LUCENE-9965: Add tooling to introspect query execution time (#144)

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (9bc5823)
            \
             N -- N -- N   refs/heads/exp (8752057)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java   | 5 +++++
 1 file changed, 5 insertions(+)

[lucene-solr] 01/01: LUCENE-9965: Add tooling to introspect query execution time (#144)

Posted by ju...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit 8752057e8d3153aace59f3f2e853090534189f64
Author: Jack Conradson <os...@gmail.com>
AuthorDate: Wed Jun 9 13:25:15 2021 -0700

    LUCENE-9965: Add tooling to introspect query execution time (#144)
    
    This change adds new IndexSearcher and Collector implementations to profile
    search execution and break down the timings. The breakdown includes the total
    time spent in each of the following categories along with the number of times
    visited: create weight, build scorer, next doc, advance, score, match.
    
    Co-authored-by: Julie Tibshirani <ju...@gmail.com>
---
 lucene/CHANGES.txt                                 |  27 +++
 .../lucene/sandbox/search/ProfilerCollector.java   | 103 +++++++++
 .../sandbox/search/ProfilerCollectorResult.java    |  67 ++++++
 .../sandbox/search/ProfilerCollectorWrapper.java   |  86 ++++++++
 .../sandbox/search/QueryProfilerBreakdown.java     |  62 ++++++
 .../sandbox/search/QueryProfilerIndexSearcher.java |  78 +++++++
 .../lucene/sandbox/search/QueryProfilerResult.java |  82 ++++++++
 .../lucene/sandbox/search/QueryProfilerScorer.java | 207 ++++++++++++++++++
 .../lucene/sandbox/search/QueryProfilerTimer.java  |  97 +++++++++
 .../sandbox/search/QueryProfilerTimingType.java    |  38 ++++
 .../lucene/sandbox/search/QueryProfilerTree.java   | 225 ++++++++++++++++++++
 .../lucene/sandbox/search/QueryProfilerWeight.java | 118 +++++++++++
 .../sandbox/search/TestProfilerCollector.java      |  67 ++++++
 .../search/TestQueryProfilerIndexSearcher.java     | 234 +++++++++++++++++++++
 .../sandbox/search/TestQueryProfilerScorer.java    |  93 ++++++++
 15 files changed, 1584 insertions(+)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 49ab6a8..0496e53 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -29,6 +29,33 @@ Other
 ---------------------
 (No changes)
 
+======================= Lucene 8.10.0 =======================
+
+API Changes
+---------------------
+(No changes)
+
+New Features
+---------------------
+(No changes)
+
+Improvements
+---------------------
+* LUCENE-9965: Added QueryProfilerIndexSearcher and ProfilerCollector to support debugging
+  query execution strategy and timing. (Jack Conradson, Julie Tibshirani)
+
+Optimizations
+---------------------
+(No changes)
+
+Bug Fixes
+---------------------
+(No changes)
+
+Other
+---------------------
+(No changes)
+
 ======================= Lucene 8.9.0 =======================
 
 API Changes
diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/ProfilerCollector.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/ProfilerCollector.java
new file mode 100644
index 0000000..328d0ff
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/ProfilerCollector.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.sandbox.search;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.LeafCollector;
+import org.apache.lucene.search.ScoreMode;
+
+/**
+ * This class wraps a Collector and times the execution of: - setScorer() - collect() -
+ * doSetNextReader() - needsScores()
+ *
+ * <p>QueryProfiler facilitates the linking of the Collector graph
+ */
+public class ProfilerCollector implements Collector {
+
+  /** A more friendly representation of the Collector's class name */
+  private final String collectorName;
+
+  /** A "hint" to help provide some context about this Collector */
+  private final String reason;
+
+  /** The wrapped collector */
+  private final ProfilerCollectorWrapper collector;
+
+  /** A list of "embedded" children collectors */
+  private final List<ProfilerCollector> children;
+
+  public ProfilerCollector(Collector collector, String reason, List<ProfilerCollector> children) {
+    this.collector = new ProfilerCollectorWrapper(collector);
+    this.reason = reason;
+    this.collectorName = deriveCollectorName(collector);
+    this.children = children;
+  }
+
+  /** @return the profiled time for this collector (inclusive of children) */
+  public long getTime() {
+    return collector.getTime();
+  }
+
+  /** @return a human readable "hint" about what this collector was used for */
+  public String getReason() {
+    return this.reason;
+  }
+
+  /** @return the lucene class name of the collector */
+  public String getName() {
+    return this.collectorName;
+  }
+
+  /**
+   * Creates a human-friendly representation of the Collector name.
+   *
+   * @param c The Collector to derive a name from
+   * @return A (hopefully) prettier name
+   */
+  private String deriveCollectorName(Collector c) {
+    return c.getClass().getSimpleName();
+  }
+
+  @Override
+  public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+    return collector.getLeafCollector(context);
+  }
+
+  @Override
+  public ScoreMode scoreMode() {
+    return collector.scoreMode();
+  }
+
+  public ProfilerCollectorResult getProfileResult() {
+    return ProfilerCollector.doGetCollectorTree(this);
+  }
+
+  private static ProfilerCollectorResult doGetCollectorTree(ProfilerCollector collector) {
+    List<ProfilerCollectorResult> childResults = new ArrayList<>(collector.children.size());
+    for (ProfilerCollector child : collector.children) {
+      ProfilerCollectorResult result = doGetCollectorTree(child);
+      childResults.add(result);
+    }
+    return new ProfilerCollectorResult(
+        collector.getName(), collector.getReason(), collector.getTime(), childResults);
+  }
+}
diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/ProfilerCollectorResult.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/ProfilerCollectorResult.java
new file mode 100644
index 0000000..0f6ec57
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/ProfilerCollectorResult.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.sandbox.search;
+
+import java.util.List;
+
+/**
+ * Public class for profiled timings of the Collectors used in the search. Children
+ * CollectorResult's may be embedded inside of a parent CollectorResult
+ */
+public class ProfilerCollectorResult {
+
+  /** A more friendly representation of the Collector's class name */
+  private final String collectorName;
+
+  /** A "hint" to help provide some context about this Collector */
+  private final String reason;
+
+  /** The total elapsed time for this Collector */
+  private final long time;
+
+  /** A list of children collectors "embedded" inside this collector */
+  private final List<ProfilerCollectorResult> children;
+
+  public ProfilerCollectorResult(
+      String collectorName, String reason, Long time, List<ProfilerCollectorResult> children) {
+    this.collectorName = collectorName;
+    this.reason = reason;
+    this.time = time;
+    this.children = children;
+  }
+
+  /** @return the profiled time for this collector (inclusive of children) */
+  public long getTime() {
+    return this.time;
+  }
+
+  /** @return a human readable "hint" about what this collector was used for */
+  public String getReason() {
+    return this.reason;
+  }
+
+  /** @return the lucene class name of the collector */
+  public String getName() {
+    return this.collectorName;
+  }
+
+  /** @return a list of children collectors */
+  public List<ProfilerCollectorResult> getProfiledChildren() {
+    return children;
+  }
+}
diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/ProfilerCollectorWrapper.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/ProfilerCollectorWrapper.java
new file mode 100644
index 0000000..3f5e28e
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/ProfilerCollectorWrapper.java
@@ -0,0 +1,86 @@
+/*
+ * 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.sandbox.search;
+
+import java.io.IOException;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.Collector;
+import org.apache.lucene.search.FilterCollector;
+import org.apache.lucene.search.FilterLeafCollector;
+import org.apache.lucene.search.LeafCollector;
+import org.apache.lucene.search.Scorable;
+import org.apache.lucene.search.ScoreMode;
+
+/** A collector that profiles how much time is spent calling it. */
+class ProfilerCollectorWrapper extends FilterCollector {
+
+  private long time;
+
+  /** Sole constructor. */
+  ProfilerCollectorWrapper(Collector in) {
+    super(in);
+  }
+
+  @Override
+  public ScoreMode scoreMode() {
+    final long start = System.nanoTime();
+    try {
+      return super.scoreMode();
+    } finally {
+      time += Math.max(1, System.nanoTime() - start);
+    }
+  }
+
+  @Override
+  public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException {
+    final long start = System.nanoTime();
+    final LeafCollector inLeafCollector;
+    try {
+      inLeafCollector = super.getLeafCollector(context);
+    } finally {
+      time += Math.max(1, System.nanoTime() - start);
+    }
+    return new FilterLeafCollector(inLeafCollector) {
+
+      @Override
+      public void collect(int doc) throws IOException {
+        final long start = System.nanoTime();
+        try {
+          super.collect(doc);
+        } finally {
+          time += Math.max(1, System.nanoTime() - start);
+        }
+      }
+
+      @Override
+      public void setScorer(Scorable scorer) throws IOException {
+        final long start = System.nanoTime();
+        try {
+          super.setScorer(scorer);
+        } finally {
+          time += Math.max(1, System.nanoTime() - start);
+        }
+      }
+    };
+  }
+
+  /** Return the total time spent on this collector. */
+  public long getTime() {
+    return time;
+  }
+}
diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerBreakdown.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerBreakdown.java
new file mode 100644
index 0000000..895ac00
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerBreakdown.java
@@ -0,0 +1,62 @@
+/*
+ * 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.sandbox.search;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * A record of timings for the various operations that may happen during query execution. A node's
+ * time may be composed of several internal attributes (rewriting, weighting, scoring, etc).
+ */
+class QueryProfilerBreakdown {
+
+  /** The accumulated timings for this query node */
+  private final QueryProfilerTimer[] timers;
+
+  /** Sole constructor. */
+  public QueryProfilerBreakdown() {
+    timers = new QueryProfilerTimer[QueryProfilerTimingType.values().length];
+    for (int i = 0; i < timers.length; ++i) {
+      timers[i] = new QueryProfilerTimer();
+    }
+  }
+
+  public QueryProfilerTimer getTimer(QueryProfilerTimingType type) {
+    return timers[type.ordinal()];
+  }
+
+  /** Build a timing count breakdown. */
+  public final Map<String, Long> toBreakdownMap() {
+    Map<String, Long> map = new HashMap<>(timers.length * 2);
+    for (QueryProfilerTimingType type : QueryProfilerTimingType.values()) {
+      map.put(type.toString(), timers[type.ordinal()].getApproximateTiming());
+      map.put(type.toString() + "_count", timers[type.ordinal()].getCount());
+    }
+    return Collections.unmodifiableMap(map);
+  }
+
+  public final long toTotalTime() {
+    long total = 0;
+    for (QueryProfilerTimer timer : timers) {
+      total += timer.getApproximateTiming();
+    }
+    return total;
+  }
+}
diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerIndexSearcher.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerIndexSearcher.java
new file mode 100644
index 0000000..71cc2f4
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerIndexSearcher.java
@@ -0,0 +1,78 @@
+/*
+ * 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.sandbox.search;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreMode;
+import org.apache.lucene.search.Weight;
+
+/**
+ * An extension of {@link IndexSearcher} that records profile information for all queries it
+ * executes.
+ */
+public class QueryProfilerIndexSearcher extends IndexSearcher {
+
+  private final QueryProfilerTree profiler;
+
+  public QueryProfilerIndexSearcher(IndexReader reader) {
+    super(reader);
+    profiler = new QueryProfilerTree();
+  }
+
+  @Override
+  public Query rewrite(Query original) throws IOException {
+    profiler.startRewriteTime();
+    try {
+      return super.rewrite(original);
+    } finally {
+      profiler.stopAndAddRewriteTime();
+    }
+  }
+
+  @Override
+  public Weight createWeight(Query query, ScoreMode scoreMode, float boost) throws IOException {
+    // createWeight() is called for each query in the tree, so we tell the queryProfiler
+    // each invocation so that it can build an internal representation of the query
+    // tree
+    QueryProfilerBreakdown profile = profiler.getProfileBreakdown(query);
+    QueryProfilerTimer timer = profile.getTimer(QueryProfilerTimingType.CREATE_WEIGHT);
+    timer.start();
+    final Weight weight;
+    try {
+      weight = query.createWeight(this, scoreMode, boost);
+    } finally {
+      timer.stop();
+      profiler.pollLast();
+    }
+    return new QueryProfilerWeight(query, weight, profile);
+  }
+
+  /** @return total time taken to rewrite all queries in this profile */
+  public long getRewriteTime() {
+    return profiler.getRewriteTime();
+  }
+
+  /** @return a hierarchical representation of the profiled tree */
+  public List<QueryProfilerResult> getProfileResult() {
+    return profiler.getTree();
+  }
+}
diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerResult.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerResult.java
new file mode 100644
index 0000000..eb3f7ac
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerResult.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.sandbox.search;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * This class is the internal representation of a profiled Query, corresponding to a single node in
+ * the query tree. It is built after the query has finished executing and is merely a structured
+ * representation, rather than the entity that collects the timing profile.
+ *
+ * <p>Each QueryProfilerResult has a List of QueryProfilerResult, which will contain "children"
+ * queries if applicable
+ */
+public class QueryProfilerResult {
+
+  private final String type;
+  private final String description;
+  private final Map<String, Long> breakdown;
+  private final long totalTime;
+  private final List<QueryProfilerResult> children;
+
+  public QueryProfilerResult(
+      String type,
+      String description,
+      Map<String, Long> breakdown,
+      long totalTime,
+      List<QueryProfilerResult> children) {
+    this.type = type;
+    this.description = description;
+    this.breakdown = Objects.requireNonNull(breakdown, "required breakdown argument missing");
+    this.children = children == null ? Collections.emptyList() : children;
+    this.totalTime = totalTime;
+  }
+
+  /** Retrieve the lucene description of this query (e.g. the "explain" text) */
+  public String getDescription() {
+    return description;
+  }
+
+  /** Retrieve the name of the entry (e.g. "TermQuery" or "LongTermsAggregator") */
+  public String getQueryName() {
+    return type;
+  }
+
+  /** The timing breakdown for this node. */
+  public Map<String, Long> getTimeBreakdown() {
+    return Collections.unmodifiableMap(breakdown);
+  }
+
+  /**
+   * Returns the total time (inclusive of children) for this query node.
+   *
+   * @return elapsed time in nanoseconds
+   */
+  public long getTotalTime() {
+    return totalTime;
+  }
+
+  /** Returns a list of all profiled children queries */
+  public List<QueryProfilerResult> getProfiledChildren() {
+    return Collections.unmodifiableList(children);
+  }
+}
diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerScorer.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerScorer.java
new file mode 100644
index 0000000..0fbdf05
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerScorer.java
@@ -0,0 +1,207 @@
+/*
+ * 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.sandbox.search;
+
+import java.io.IOException;
+import java.util.Collection;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TwoPhaseIterator;
+import org.apache.lucene.search.Weight;
+
+/**
+ * {@link Scorer} wrapper that will compute how much time is spent on moving the iterator,
+ * confirming matches and computing scores.
+ */
+class QueryProfilerScorer extends Scorer {
+
+  private final Scorer scorer;
+  private QueryProfilerWeight profileWeight;
+
+  private final QueryProfilerTimer scoreTimer,
+      nextDocTimer,
+      advanceTimer,
+      matchTimer,
+      shallowAdvanceTimer,
+      computeMaxScoreTimer,
+      setMinCompetitiveScoreTimer;
+
+  QueryProfilerScorer(QueryProfilerWeight w, Scorer scorer, QueryProfilerBreakdown profile) {
+    super(w);
+    this.scorer = scorer;
+    this.profileWeight = w;
+    scoreTimer = profile.getTimer(QueryProfilerTimingType.SCORE);
+    nextDocTimer = profile.getTimer(QueryProfilerTimingType.NEXT_DOC);
+    advanceTimer = profile.getTimer(QueryProfilerTimingType.ADVANCE);
+    matchTimer = profile.getTimer(QueryProfilerTimingType.MATCH);
+    shallowAdvanceTimer = profile.getTimer(QueryProfilerTimingType.SHALLOW_ADVANCE);
+    computeMaxScoreTimer = profile.getTimer(QueryProfilerTimingType.COMPUTE_MAX_SCORE);
+    setMinCompetitiveScoreTimer =
+        profile.getTimer(QueryProfilerTimingType.SET_MIN_COMPETITIVE_SCORE);
+  }
+
+  @Override
+  public int docID() {
+    return scorer.docID();
+  }
+
+  @Override
+  public float score() throws IOException {
+    scoreTimer.start();
+    try {
+      return scorer.score();
+    } finally {
+      scoreTimer.stop();
+    }
+  }
+
+  @Override
+  public Weight getWeight() {
+    return profileWeight;
+  }
+
+  @Override
+  public Collection<ChildScorable> getChildren() throws IOException {
+    return scorer.getChildren();
+  }
+
+  @Override
+  public DocIdSetIterator iterator() {
+    final DocIdSetIterator in = scorer.iterator();
+    return new DocIdSetIterator() {
+
+      @Override
+      public int advance(int target) throws IOException {
+        advanceTimer.start();
+        try {
+          return in.advance(target);
+        } finally {
+          advanceTimer.stop();
+        }
+      }
+
+      @Override
+      public int nextDoc() throws IOException {
+        nextDocTimer.start();
+        try {
+          return in.nextDoc();
+        } finally {
+          nextDocTimer.stop();
+        }
+      }
+
+      @Override
+      public int docID() {
+        return in.docID();
+      }
+
+      @Override
+      public long cost() {
+        return in.cost();
+      }
+    };
+  }
+
+  @Override
+  public TwoPhaseIterator twoPhaseIterator() {
+    final TwoPhaseIterator in = scorer.twoPhaseIterator();
+    if (in == null) {
+      return null;
+    }
+    final DocIdSetIterator inApproximation = in.approximation();
+    final DocIdSetIterator approximation =
+        new DocIdSetIterator() {
+
+          @Override
+          public int advance(int target) throws IOException {
+            advanceTimer.start();
+            try {
+              return inApproximation.advance(target);
+            } finally {
+              advanceTimer.stop();
+            }
+          }
+
+          @Override
+          public int nextDoc() throws IOException {
+            nextDocTimer.start();
+            try {
+              return inApproximation.nextDoc();
+            } finally {
+              nextDocTimer.stop();
+            }
+          }
+
+          @Override
+          public int docID() {
+            return inApproximation.docID();
+          }
+
+          @Override
+          public long cost() {
+            return inApproximation.cost();
+          }
+        };
+    return new TwoPhaseIterator(approximation) {
+      @Override
+      public boolean matches() throws IOException {
+        matchTimer.start();
+        try {
+          return in.matches();
+        } finally {
+          matchTimer.stop();
+        }
+      }
+
+      @Override
+      public float matchCost() {
+        return in.matchCost();
+      }
+    };
+  }
+
+  @Override
+  public int advanceShallow(int target) throws IOException {
+    shallowAdvanceTimer.start();
+    try {
+      return scorer.advanceShallow(target);
+    } finally {
+      shallowAdvanceTimer.stop();
+    }
+  }
+
+  @Override
+  public float getMaxScore(int upTo) throws IOException {
+    computeMaxScoreTimer.start();
+    try {
+      return scorer.getMaxScore(upTo);
+    } finally {
+      computeMaxScoreTimer.stop();
+    }
+  }
+
+  @Override
+  public void setMinCompetitiveScore(float minScore) throws IOException {
+    setMinCompetitiveScoreTimer.start();
+    try {
+      scorer.setMinCompetitiveScore(minScore);
+    } finally {
+      setMinCompetitiveScoreTimer.stop();
+    }
+  }
+}
diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerTimer.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerTimer.java
new file mode 100644
index 0000000..1d2784f
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerTimer.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.sandbox.search;
+
+/**
+ * Helps measure how much time is spent running some methods. The {@link #start()} and {@link
+ * #stop()} methods should typically be called in a try/finally clause with {@link #start()} being
+ * called right before the try block and {@link #stop()} being called at the beginning of the
+ * finally block:
+ *
+ * <pre>
+ *  timer.start();
+ *  try {
+ *    // code to time
+ *  } finally {
+ *    timer.stop();
+ *  }
+ *  </pre>
+ */
+class QueryProfilerTimer {
+
+  private boolean doTiming;
+  private long timing, count, lastCount, start;
+
+  /** pkg-private for testing */
+  long nanoTime() {
+    return System.nanoTime();
+  }
+
+  /** Start the timer. */
+  public final void start() {
+    assert start == 0 : "#start call misses a matching #stop call";
+    // We measure the timing of each method call for the first 256
+    // calls, then 1/2 call up to 512 then 1/3 up to 768, etc. with
+    // a maximum interval of 1024, which is reached for 1024*2^8 ~= 262000
+    // This allows to not slow down things too much because of calls
+    // to System.nanoTime() when methods are called millions of time
+    // in tight loops, while still providing useful timings for methods
+    // that are only called a couple times per search execution.
+    doTiming = (count - lastCount) >= Math.min(lastCount >>> 8, 1024);
+    if (doTiming) {
+      start = nanoTime();
+    }
+    count++;
+  }
+
+  /** Stop the timer. */
+  public final void stop() {
+    if (doTiming) {
+      timing += (count - lastCount) * Math.max(nanoTime() - start, 1L);
+      lastCount = count;
+      start = 0;
+    }
+  }
+
+  /** Return the number of times that {@link #start()} has been called. */
+  public final long getCount() {
+    if (start != 0) {
+      throw new IllegalStateException("#start call misses a matching #stop call");
+    }
+    return count;
+  }
+
+  /**
+   * Return an approximation of the total time spent between consecutive calls of #start and #stop.
+   */
+  public final long getApproximateTiming() {
+    if (start != 0) {
+      throw new IllegalStateException("#start call misses a matching #stop call");
+    }
+    // We don't have timings for the last `count-lastCount` method calls
+    // so we assume that they had the same timing as the lastCount first
+    // calls. This approximation is ok since at most 1/256th of method
+    // calls have not been timed.
+    long timing = this.timing;
+    if (count > lastCount) {
+      assert lastCount > 0;
+      timing += (count - lastCount) * timing / lastCount;
+    }
+    return timing;
+  }
+}
diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerTimingType.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerTimingType.java
new file mode 100644
index 0000000..ebde56d
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerTimingType.java
@@ -0,0 +1,38 @@
+/*
+ * 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.sandbox.search;
+
+import java.util.Locale;
+
+/** This enum breaks down the query into different sections to describe what was timed. */
+public enum QueryProfilerTimingType {
+  CREATE_WEIGHT,
+  BUILD_SCORER,
+  NEXT_DOC,
+  ADVANCE,
+  MATCH,
+  SCORE,
+  SHALLOW_ADVANCE,
+  COMPUTE_MAX_SCORE,
+  SET_MIN_COMPETITIVE_SCORE;
+
+  @Override
+  public String toString() {
+    return name().toLowerCase(Locale.ROOT);
+  }
+}
diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerTree.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerTree.java
new file mode 100644
index 0000000..8a0da1f
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerTree.java
@@ -0,0 +1,225 @@
+/*
+ * 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.sandbox.search;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.List;
+import org.apache.lucene.search.Query;
+
+/**
+ * This class tracks the dependency tree for queries (scoring and rewriting) and generates {@link
+ * QueryProfilerBreakdown} for each node in the tree.
+ */
+class QueryProfilerTree {
+
+  private final ArrayList<QueryProfilerBreakdown> breakdowns;
+  /** Maps the Query to it's list of children. This is basically the dependency tree */
+  private final ArrayList<ArrayList<Integer>> tree;
+  /** A list of the original queries, keyed by index position */
+  private final ArrayList<Query> queries;
+  /** A list of top-level "roots". Each root can have its own tree of profiles */
+  private final ArrayList<Integer> roots;
+  /** A temporary stack used to record where we are in the dependency tree. */
+  private final Deque<Integer> stack;
+
+  private int currentToken = 0;
+
+  /** Rewrite time */
+  private long rewriteTime;
+
+  private long rewriteScratch;
+
+  public QueryProfilerTree() {
+    breakdowns = new ArrayList<>(10);
+    stack = new ArrayDeque<>(10);
+    tree = new ArrayList<>(10);
+    queries = new ArrayList<>(10);
+    roots = new ArrayList<>(10);
+  }
+
+  /**
+   * Returns a {@link QueryProfilerBreakdown} for a scoring query. Scoring queries (e.g. those that
+   * are past the rewrite phase and are now being wrapped by createWeight() ) follow a recursive
+   * progression. We can track the dependency tree by a simple stack
+   *
+   * <p>The only hiccup is that the first scoring query will be identical to the last rewritten
+   * query, so we need to take special care to fix that
+   *
+   * @param query The scoring query we wish to profile
+   * @return A ProfileBreakdown for this query
+   */
+  public QueryProfilerBreakdown getProfileBreakdown(Query query) {
+    int token = currentToken;
+
+    boolean stackEmpty = stack.isEmpty();
+
+    // If the stack is empty, we are a new root query
+    if (stackEmpty) {
+
+      // We couldn't find a rewritten query to attach to, so just add it as a
+      // top-level root. This is just a precaution: it really shouldn't happen.
+      // We would only get here if a top-level query that never rewrites for some reason.
+      roots.add(token);
+
+      // Increment the token since we are adding a new node, but notably, do not
+      // updateParent() because this was added as a root
+      currentToken += 1;
+      stack.add(token);
+
+      return addDependencyNode(query, token);
+    }
+
+    updateParent(token);
+
+    // Increment the token since we are adding a new node
+    currentToken += 1;
+    stack.add(token);
+
+    return addDependencyNode(query, token);
+  }
+
+  /**
+   * Helper method to add a new node to the dependency tree.
+   *
+   * <p>Initializes a new list in the dependency tree, saves the query and generates a new {@link
+   * QueryProfilerBreakdown} to track the timings of this query.
+   *
+   * @param query The query to profile
+   * @param token The assigned token for this query
+   * @return A {@link QueryProfilerBreakdown} to profile this query
+   */
+  private QueryProfilerBreakdown addDependencyNode(Query query, int token) {
+
+    // Add a new slot in the dependency tree
+    tree.add(new ArrayList<>(5));
+
+    // Save our query for lookup later
+    queries.add(query);
+
+    QueryProfilerBreakdown breakdown = createProfileBreakdown();
+    breakdowns.add(token, breakdown);
+    return breakdown;
+  }
+
+  private QueryProfilerBreakdown createProfileBreakdown() {
+    return new QueryProfilerBreakdown();
+  }
+
+  /** Removes the last (e.g. most recent) value on the stack */
+  public void pollLast() {
+    stack.pollLast();
+  }
+
+  /**
+   * After the query has been run and profiled, we need to merge the flat timing map with the
+   * dependency graph to build a data structure that mirrors the original query tree
+   *
+   * @return a hierarchical representation of the profiled query tree
+   */
+  public List<QueryProfilerResult> getTree() {
+    ArrayList<QueryProfilerResult> results = new ArrayList<>(roots.size());
+    for (Integer root : roots) {
+      results.add(doGetTree(root));
+    }
+    return results;
+  }
+
+  /**
+   * Recursive helper to finalize a node in the dependency tree
+   *
+   * @param token The node we are currently finalizing
+   * @return A hierarchical representation of the tree inclusive of children at this level
+   */
+  private QueryProfilerResult doGetTree(int token) {
+    Query query = queries.get(token);
+    QueryProfilerBreakdown breakdown = breakdowns.get(token);
+    List<Integer> children = tree.get(token);
+    List<QueryProfilerResult> childrenProfileResults = Collections.emptyList();
+
+    if (children != null) {
+      childrenProfileResults = new ArrayList<>(children.size());
+      for (Integer child : children) {
+        QueryProfilerResult childNode = doGetTree(child);
+        childrenProfileResults.add(childNode);
+      }
+    }
+
+    // TODO this would be better done bottom-up instead of top-down to avoid
+    // calculating the same times over and over...but worth the effort?
+    String type = getTypeFromQuery(query);
+    String description = getDescriptionFromQuery(query);
+    return new QueryProfilerResult(
+        type,
+        description,
+        breakdown.toBreakdownMap(),
+        breakdown.toTotalTime(),
+        childrenProfileResults);
+  }
+
+  private String getTypeFromQuery(Query query) {
+    // Anonymous classes won't have a name,
+    // we need to get the super class
+    if (query.getClass().getSimpleName().isEmpty()) {
+      return query.getClass().getSuperclass().getSimpleName();
+    }
+    return query.getClass().getSimpleName();
+  }
+
+  private String getDescriptionFromQuery(Query query) {
+    return query.toString();
+  }
+
+  /**
+   * Internal helper to add a child to the current parent node
+   *
+   * @param childToken The child to add to the current parent
+   */
+  private void updateParent(int childToken) {
+    Integer parent = stack.peekLast();
+    ArrayList<Integer> parentNode = tree.get(parent);
+    parentNode.add(childToken);
+    tree.set(parent, parentNode);
+  }
+
+  /** Begin timing a query for a specific Timing context */
+  public void startRewriteTime() {
+    assert rewriteScratch == 0;
+    rewriteScratch = System.nanoTime();
+  }
+
+  /**
+   * Halt the timing process and add the elapsed rewriting time. startRewriteTime() must be called
+   * for a particular context prior to calling stopAndAddRewriteTime(), otherwise the elapsed time
+   * will be negative and nonsensical
+   *
+   * @return The elapsed time
+   */
+  public long stopAndAddRewriteTime() {
+    long time = Math.max(1, System.nanoTime() - rewriteScratch);
+    rewriteTime += time;
+    rewriteScratch = 0;
+    return time;
+  }
+
+  public long getRewriteTime() {
+    return rewriteTime;
+  }
+}
diff --git a/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java
new file mode 100644
index 0000000..f3a0976
--- /dev/null
+++ b/lucene/sandbox/src/java/org/apache/lucene/sandbox/search/QueryProfilerWeight.java
@@ -0,0 +1,118 @@
+/*
+ * 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.sandbox.search;
+
+import java.io.IOException;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.BulkScorer;
+import org.apache.lucene.search.Explanation;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.ScorerSupplier;
+import org.apache.lucene.search.Weight;
+
+/**
+ * Weight wrapper that will compute how much time it takes to build the {@link Scorer} and then
+ * return a {@link Scorer} that is wrapped in order to compute timings as well.
+ */
+class QueryProfilerWeight extends Weight {
+
+  private final Weight subQueryWeight;
+  private final QueryProfilerBreakdown profile;
+
+  public QueryProfilerWeight(Query query, Weight subQueryWeight, QueryProfilerBreakdown profile) {
+    super(query);
+    this.subQueryWeight = subQueryWeight;
+    this.profile = profile;
+  }
+
+  @Override
+  public Scorer scorer(LeafReaderContext context) throws IOException {
+    ScorerSupplier supplier = scorerSupplier(context);
+    if (supplier == null) {
+      return null;
+    }
+    return supplier.get(Long.MAX_VALUE);
+  }
+
+  @Override
+  public ScorerSupplier scorerSupplier(LeafReaderContext context) throws IOException {
+    QueryProfilerTimer timer = profile.getTimer(QueryProfilerTimingType.BUILD_SCORER);
+    timer.start();
+    final ScorerSupplier subQueryScorerSupplier;
+    try {
+      subQueryScorerSupplier = subQueryWeight.scorerSupplier(context);
+    } finally {
+      timer.stop();
+    }
+    if (subQueryScorerSupplier == null) {
+      return null;
+    }
+
+    final QueryProfilerWeight weight = this;
+    return new ScorerSupplier() {
+
+      @Override
+      public Scorer get(long loadCost) throws IOException {
+        timer.start();
+        try {
+          return new QueryProfilerScorer(weight, subQueryScorerSupplier.get(loadCost), profile);
+        } finally {
+          timer.stop();
+        }
+      }
+
+      @Override
+      public long cost() {
+        timer.start();
+        try {
+          return subQueryScorerSupplier.cost();
+        } finally {
+          timer.stop();
+        }
+      }
+    };
+  }
+
+  @Override
+  public BulkScorer bulkScorer(LeafReaderContext context) throws IOException {
+    // We use the default bulk scorer instead of the specialized one. The reason
+    // is that Lucene's BulkScorers do everything at once: finding matches,
+    // scoring them and calling the collector, so they make it impossible to
+    // see where time is spent, which is the purpose of query profiling.
+    // The default bulk scorer will pull a scorer and iterate over matches,
+    // this might be a significantly different execution path for some queries
+    // like disjunctions, but in general this is what is done anyway
+    return super.bulkScorer(context);
+  }
+
+  @Override
+  public Explanation explain(LeafReaderContext context, int doc) throws IOException {
+    return subQueryWeight.explain(context, doc);
+  }
+
+  @Override
+  public boolean isCacheable(LeafReaderContext ctx) {
+    return false;
+  }
+
+  @Override
+  public void extractTerms(Set<Term> terms) {
+    subQueryWeight.extractTerms(terms);
+  }
+}
diff --git a/lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestProfilerCollector.java b/lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestProfilerCollector.java
new file mode 100644
index 0000000..3d68ee1
--- /dev/null
+++ b/lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestProfilerCollector.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.sandbox.search;
+
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThan;
+
+import java.io.IOException;
+import java.util.List;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.TotalHitCountCollector;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.hamcrest.MatcherAssert;
+
+public class TestProfilerCollector extends LuceneTestCase {
+
+  public void testCollector() throws IOException {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    final int numDocs = TestUtil.nextInt(random(), 1, 20);
+    for (int i = 0; i < numDocs; ++i) {
+      Document doc = new Document();
+      doc.add(new StringField("foo", "bar", Store.NO));
+      w.addDocument(doc);
+    }
+    IndexReader reader = w.getReader();
+    w.close();
+
+    ProfilerCollector collector =
+        new ProfilerCollector(new TotalHitCountCollector(), "total_hits", List.of());
+    IndexSearcher searcher = new IndexSearcher(reader);
+    Query query = new TermQuery(new Term("foo", "bar"));
+    searcher.search(query, collector);
+
+    ProfilerCollectorResult profileResult = collector.getProfileResult();
+    MatcherAssert.assertThat(profileResult.getReason(), equalTo("total_hits"));
+    MatcherAssert.assertThat(profileResult.getTime(), greaterThan(0L));
+
+    reader.close();
+    dir.close();
+  }
+}
diff --git a/lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestQueryProfilerIndexSearcher.java b/lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestQueryProfilerIndexSearcher.java
new file mode 100644
index 0000000..a42cb56
--- /dev/null
+++ b/lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestQueryProfilerIndexSearcher.java
@@ -0,0 +1,234 @@
+/*
+ * 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.sandbox.search;
+
+import static org.hamcrest.Matchers.equalTo;
+import static org.hamcrest.Matchers.greaterThan;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field.Store;
+import org.apache.lucene.document.StringField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.LRUQueryCache;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.RandomApproximationQuery;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.hamcrest.MatcherAssert;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+public class TestQueryProfilerIndexSearcher extends LuceneTestCase {
+
+  private static Directory dir;
+  private static IndexReader reader;
+
+  @BeforeClass
+  public static void setup() throws IOException {
+    dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    final int numDocs = TestUtil.nextInt(random(), 1, 20);
+    for (int i = 0; i < numDocs; ++i) {
+      final int numHoles = random().nextInt(5);
+      for (int j = 0; j < numHoles; ++j) {
+        w.addDocument(new Document());
+      }
+      Document doc = new Document();
+      doc.add(new StringField("foo", "bar", Store.NO));
+      w.addDocument(doc);
+    }
+    reader = w.getReader();
+    w.close();
+  }
+
+  @AfterClass
+  public static void cleanup() throws IOException {
+    IOUtils.close(reader, dir);
+    dir = null;
+    reader = null;
+  }
+
+  public void testBasic() throws IOException {
+    QueryProfilerIndexSearcher searcher = new QueryProfilerIndexSearcher(reader);
+    Query query = new TermQuery(new Term("foo", "bar"));
+    searcher.search(query, 1);
+
+    List<QueryProfilerResult> results = searcher.getProfileResult();
+    assertEquals(1, results.size());
+    Map<String, Long> breakdown = results.get(0).getTimeBreakdown();
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.CREATE_WEIGHT.toString()), greaterThan(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.BUILD_SCORER.toString()), greaterThan(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.NEXT_DOC.toString()), greaterThan(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.ADVANCE.toString()), equalTo(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.SCORE.toString()), greaterThan(0L));
+    MatcherAssert.assertThat(breakdown.get(QueryProfilerTimingType.MATCH.toString()), equalTo(0L));
+
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.CREATE_WEIGHT.toString() + "_count"),
+        greaterThan(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.BUILD_SCORER.toString() + "_count"), greaterThan(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.NEXT_DOC.toString() + "_count"), greaterThan(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.ADVANCE.toString() + "_count"), equalTo(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.SCORE.toString() + "_count"), greaterThan(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.MATCH.toString() + "_count"), equalTo(0L));
+
+    long rewriteTime = searcher.getRewriteTime();
+    MatcherAssert.assertThat(rewriteTime, greaterThan(0L));
+  }
+
+  public void testTwoQueries() throws IOException {
+    QueryProfilerIndexSearcher searcher = new QueryProfilerIndexSearcher(reader);
+    Query firstQuery = new TermQuery(new Term("foo", "bar"));
+    searcher.search(firstQuery, 1);
+
+    Query secondQuery = new TermQuery(new Term("foo", "baz"));
+    searcher.search(secondQuery, 1);
+
+    List<QueryProfilerResult> results = searcher.getProfileResult();
+    assertEquals(2, results.size());
+
+    Map<String, Long> firstResult = results.get(0).getTimeBreakdown();
+    MatcherAssert.assertThat(
+        firstResult.get(QueryProfilerTimingType.CREATE_WEIGHT.toString()), greaterThan(0L));
+
+    Map<String, Long> secondResult = results.get(1).getTimeBreakdown();
+    MatcherAssert.assertThat(
+        secondResult.get(QueryProfilerTimingType.CREATE_WEIGHT.toString()), greaterThan(0L));
+
+    long rewriteTime = searcher.getRewriteTime();
+    MatcherAssert.assertThat(rewriteTime, greaterThan(0L));
+  }
+
+  public void testNoCaching() throws IOException {
+    IndexSearcher searcher = new QueryProfilerIndexSearcher(reader);
+    Query query = new TermQuery(new Term("foo", "bar"));
+    searcher.search(query, 1);
+
+    LRUQueryCache cache = (LRUQueryCache) searcher.getQueryCache();
+    MatcherAssert.assertThat(cache.getHitCount(), equalTo(0L));
+    MatcherAssert.assertThat(cache.getCacheCount(), equalTo(0L));
+    MatcherAssert.assertThat(cache.getTotalCount(), equalTo(cache.getMissCount()));
+    MatcherAssert.assertThat(cache.getCacheSize(), equalTo(0L));
+  }
+
+  public void testNoScoring() throws IOException {
+    QueryProfilerIndexSearcher searcher = new QueryProfilerIndexSearcher(reader);
+    Query query = new TermQuery(new Term("foo", "bar"));
+    searcher.search(query, 1, Sort.INDEXORDER); // scores are not needed
+
+    List<QueryProfilerResult> results = searcher.getProfileResult();
+    assertEquals(1, results.size());
+    Map<String, Long> breakdown = results.get(0).getTimeBreakdown();
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.CREATE_WEIGHT.toString()), greaterThan(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.BUILD_SCORER.toString()), greaterThan(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.NEXT_DOC.toString()), greaterThan(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.ADVANCE.toString()), equalTo(0L));
+    MatcherAssert.assertThat(breakdown.get(QueryProfilerTimingType.SCORE.toString()), equalTo(0L));
+    MatcherAssert.assertThat(breakdown.get(QueryProfilerTimingType.MATCH.toString()), equalTo(0L));
+
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.CREATE_WEIGHT.toString() + "_count"),
+        greaterThan(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.BUILD_SCORER.toString() + "_count"), greaterThan(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.NEXT_DOC.toString() + "_count"), greaterThan(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.ADVANCE.toString() + "_count"), equalTo(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.SCORE.toString() + "_count"), equalTo(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.MATCH.toString() + "_count"), equalTo(0L));
+
+    long rewriteTime = searcher.getRewriteTime();
+    MatcherAssert.assertThat(rewriteTime, greaterThan(0L));
+  }
+
+  public void testUseIndexStats() throws IOException {
+    QueryProfilerIndexSearcher searcher = new QueryProfilerIndexSearcher(reader);
+    Query query = new TermQuery(new Term("foo", "bar"));
+    searcher.count(query); // will use index stats
+
+    List<QueryProfilerResult> results = searcher.getProfileResult();
+    assertEquals(0, results.size());
+
+    long rewriteTime = searcher.getRewriteTime();
+    MatcherAssert.assertThat(rewriteTime, greaterThan(0L));
+  }
+
+  public void testApproximations() throws IOException {
+    QueryProfilerIndexSearcher searcher = new QueryProfilerIndexSearcher(reader);
+    Query query = new RandomApproximationQuery(new TermQuery(new Term("foo", "bar")), random());
+    searcher.count(query);
+    List<QueryProfilerResult> results = searcher.getProfileResult();
+    assertEquals(1, results.size());
+    Map<String, Long> breakdown = results.get(0).getTimeBreakdown();
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.CREATE_WEIGHT.toString()), greaterThan(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.BUILD_SCORER.toString()), greaterThan(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.NEXT_DOC.toString()), greaterThan(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.ADVANCE.toString()), equalTo(0L));
+    MatcherAssert.assertThat(breakdown.get(QueryProfilerTimingType.SCORE.toString()), equalTo(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.MATCH.toString()), greaterThan(0L));
+
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.CREATE_WEIGHT.toString() + "_count"),
+        greaterThan(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.BUILD_SCORER.toString() + "_count"), greaterThan(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.NEXT_DOC.toString() + "_count"), greaterThan(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.ADVANCE.toString() + "_count"), equalTo(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.SCORE.toString() + "_count"), equalTo(0L));
+    MatcherAssert.assertThat(
+        breakdown.get(QueryProfilerTimingType.MATCH.toString() + "_count"), greaterThan(0L));
+
+    long rewriteTime = searcher.getRewriteTime();
+    MatcherAssert.assertThat(rewriteTime, greaterThan(0L));
+  }
+}
diff --git a/lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestQueryProfilerScorer.java b/lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestQueryProfilerScorer.java
new file mode 100644
index 0000000..c564781
--- /dev/null
+++ b/lucene/sandbox/src/test/org/apache/lucene/sandbox/search/TestQueryProfilerScorer.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.sandbox.search;
+
+import java.io.IOException;
+import org.apache.lucene.index.MultiReader;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.MatchAllDocsQuery;
+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.LuceneTestCase;
+
+public class TestQueryProfilerScorer extends LuceneTestCase {
+
+  private static class FakeScorer extends Scorer {
+
+    public float maxScore, minCompetitiveScore;
+
+    protected FakeScorer(Weight weight) {
+      super(weight);
+    }
+
+    @Override
+    public DocIdSetIterator iterator() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public float getMaxScore(int upTo) throws IOException {
+      return maxScore;
+    }
+
+    @Override
+    public float score() throws IOException {
+      return 1f;
+    }
+
+    @Override
+    public int docID() {
+      throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public void setMinCompetitiveScore(float minScore) {
+      this.minCompetitiveScore = minScore;
+    }
+  }
+
+  public void testPropagateMinCompetitiveScore() throws IOException {
+    Query query = new MatchAllDocsQuery();
+    Weight weight =
+        query.createWeight(new IndexSearcher(new MultiReader()), ScoreMode.TOP_SCORES, 1f);
+    FakeScorer fakeScorer = new FakeScorer(weight);
+    QueryProfilerBreakdown profile = new QueryProfilerBreakdown();
+    QueryProfilerWeight queryProfilerWeight = new QueryProfilerWeight(query, weight, profile);
+    QueryProfilerScorer queryProfilerScorer =
+        new QueryProfilerScorer(queryProfilerWeight, fakeScorer, profile);
+    queryProfilerScorer.setMinCompetitiveScore(0.42f);
+    assertEquals(0.42f, fakeScorer.minCompetitiveScore, 0f);
+  }
+
+  public void testPropagateMaxScore() throws IOException {
+    Query query = new MatchAllDocsQuery();
+    Weight weight =
+        query.createWeight(new IndexSearcher(new MultiReader()), ScoreMode.TOP_SCORES, 1f);
+    FakeScorer fakeScorer = new FakeScorer(weight);
+    QueryProfilerBreakdown profile = new QueryProfilerBreakdown();
+    QueryProfilerWeight queryProfilerWeight = new QueryProfilerWeight(query, weight, profile);
+    QueryProfilerScorer queryProfilerScorer =
+        new QueryProfilerScorer(queryProfilerWeight, fakeScorer, profile);
+    queryProfilerScorer.setMinCompetitiveScore(0.42f);
+    fakeScorer.maxScore = 42f;
+    assertEquals(42f, queryProfilerScorer.getMaxScore(DocIdSetIterator.NO_MORE_DOCS), 0f);
+  }
+}