You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@solr.apache.org by gu...@apache.org on 2024/02/12 15:07:20 UTC

(solr) branch main updated: SOLR-17140 - provide extensible query limit concept. (#2237)

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

gus pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/solr.git


The following commit(s) were added to refs/heads/main by this push:
     new 07f65499723 SOLR-17140 - provide extensible query limit concept. (#2237)
07f65499723 is described below

commit 07f65499723d3190041849a7b3e2d70593b3eda5
Author: Gus Heck <46...@users.noreply.github.com>
AuthorDate: Mon Feb 12 10:07:14 2024 -0500

    SOLR-17140 - provide extensible query limit concept. (#2237)
    
    * SOLR-17140 - provide extensible query limit concept.
    This replaces SolrQueryTimeoutImpl.java with a much simpler SolrTimeLimit and provides a QueryLimits class to allow addition of additional types of limits. (see also SOLR-17138)
    
    * SOLR-17140 ensure we always have limits. SolrRequestInfo is created by HttpSolrCall, so any code path not flowing through that needs to ensure SolrRequestInfo if it wants to use limit functionality.
    
    * SOLR-17140 Limits should not be lost if we push a new request onto the stack. This makes it difficult to change limits for sub-requests, but I'm not convinced that such a thing makes sense anyway.
---
 .../apache/solr/handler/MoreLikeThisHandler.java   |   4 -
 .../solr/handler/component/SearchHandler.java      |   4 -
 .../org/apache/solr/request/SolrRequestInfo.java   |  34 +++++++
 .../java/org/apache/solr/search/QueryLimits.java   |  95 ++++++++++++++++++
 .../org/apache/solr/search/SolrIndexSearcher.java  |  23 +++--
 .../org/apache/solr/search/SolrQueryTimeLimit.java |  66 +++++++++++++
 .../apache/solr/search/SolrQueryTimeoutImpl.java   | 107 ---------------------
 .../apache/solr/request/TestSolrRequestInfo.java   |  53 ++++++++++
 8 files changed, 263 insertions(+), 123 deletions(-)

diff --git a/solr/core/src/java/org/apache/solr/handler/MoreLikeThisHandler.java b/solr/core/src/java/org/apache/solr/handler/MoreLikeThisHandler.java
index 7ebe55355aa..4cb201b0287 100644
--- a/solr/core/src/java/org/apache/solr/handler/MoreLikeThisHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/MoreLikeThisHandler.java
@@ -65,7 +65,6 @@ import org.apache.solr.search.QueryParsing;
 import org.apache.solr.search.QueryUtils;
 import org.apache.solr.search.ReturnFields;
 import org.apache.solr.search.SolrIndexSearcher;
-import org.apache.solr.search.SolrQueryTimeoutImpl;
 import org.apache.solr.search.SolrReturnFields;
 import org.apache.solr.search.SortSpec;
 import org.apache.solr.search.SyntaxError;
@@ -97,7 +96,6 @@ public class MoreLikeThisHandler extends RequestHandlerBase {
   public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception {
     SolrParams params = req.getParams();
 
-    SolrQueryTimeoutImpl.set(req);
     try {
 
       // Set field flags
@@ -267,8 +265,6 @@ public class MoreLikeThisHandler extends RequestHandlerBase {
       }
     } catch (ExitableDirectoryReader.ExitingReaderException ex) {
       log.warn("Query: {}; ", req.getParamString(), ex);
-    } finally {
-      SolrQueryTimeoutImpl.reset();
     }
   }
 
diff --git a/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java b/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
index 92b8f090dfc..7dda2b036ad 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/SearchHandler.java
@@ -61,7 +61,6 @@ import org.apache.solr.pkg.SolrPackageLoader;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.search.CursorMark;
-import org.apache.solr.search.SolrQueryTimeoutImpl;
 import org.apache.solr.search.SortSpec;
 import org.apache.solr.search.facet.FacetModule;
 import org.apache.solr.security.AuthorizationContext;
@@ -458,7 +457,6 @@ public class SearchHandler extends RequestHandlerBase
     if (!rb.isDistrib) {
       // a normal non-distributed request
 
-      SolrQueryTimeoutImpl.set(req);
       try {
         // The semantics of debugging vs not debugging are different enough that
         // it makes sense to have two control loops
@@ -503,8 +501,6 @@ public class SearchHandler extends RequestHandlerBase
             .getResponseHeader()
             .asShallowMap()
             .put(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY, Boolean.TRUE);
-      } finally {
-        SolrQueryTimeoutImpl.reset();
       }
     } else {
       // a distributed request
diff --git a/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java b/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
index e90e33ee8fa..9c00b86a67f 100644
--- a/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
+++ b/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
@@ -31,6 +31,7 @@ import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.handler.component.ResponseBuilder;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.search.QueryLimits;
 import org.apache.solr.servlet.SolrDispatchFilter;
 import org.apache.solr.util.TimeZoneUtils;
 import org.slf4j.Logger;
@@ -43,6 +44,7 @@ public class SolrRequestInfo {
 
   private static final ThreadLocal<Deque<SolrRequestInfo>> threadLocal =
       ThreadLocal.withInitial(ArrayDeque::new);
+  static final Object LIMITS_KEY = new Object();
 
   private int refCount = 1; // prevent closing when still used
 
@@ -75,7 +77,13 @@ public class SolrRequestInfo {
     } else if (stack.size() > MAX_STACK_SIZE) {
       assert false : "SolrRequestInfo Stack is full";
       log.error("SolrRequestInfo Stack is full");
+    } else if (!stack.isEmpty() && info.req != null) {
+      // New SRI instances inherit limits from prior SRI regardless of parameters.
+      // This ensures limits cannot be changed or removed for a given thread once set.
+      // if req is null limits will be an empty instance with no limits anyway.
+      info.req.getContext().put(LIMITS_KEY, stack.peek().getLimits());
     }
+    info.initQueryLimits();
     log.trace("{} {}", info, "setRequestInfo()");
     assert !info.isClosed() : "SRI is already closed (odd).";
     stack.push(info);
@@ -210,6 +218,32 @@ public class SolrRequestInfo {
     }
   }
 
+  /**
+   * This call creates the QueryLimits object and any required implementations of {@link
+   * org.apache.lucene.index.QueryTimeout}. Any code before this call will not be subject to the
+   * limitations set on the request. Note that calling {@link #getLimits()} has the same effect as
+   * this method.
+   *
+   * @see #getLimits()
+   */
+  private void initQueryLimits() {
+    // This method only exists for code clarity reasons.
+    getLimits();
+  }
+
+  /**
+   * Get the query limits for the current request. This will trigger the creation of the (possibly
+   * empty) {@link QueryLimits} object if it has not been created, and will then return the same
+   * object on every subsequent invocation.
+   *
+   * @return The {@code QueryLimits} object for the current requet.
+   */
+  public QueryLimits getLimits() {
+    return req == null
+        ? QueryLimits.NONE
+        : (QueryLimits) req.getContext().computeIfAbsent(LIMITS_KEY, (k) -> new QueryLimits(req));
+  }
+
   public SolrDispatchFilter.Action getAction() {
     return action;
   }
diff --git a/solr/core/src/java/org/apache/solr/search/QueryLimits.java b/solr/core/src/java/org/apache/solr/search/QueryLimits.java
new file mode 100644
index 00000000000..5b04b0033a6
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/QueryLimits.java
@@ -0,0 +1,95 @@
+/*
+ * 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.solr.search;
+
+import static org.apache.solr.search.SolrQueryTimeLimit.hasTimeLimit;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.lucene.index.QueryTimeout;
+import org.apache.solr.request.SolrQueryRequest;
+
+/**
+ * Represents the limitations on the query. These limits might be wall clock time, cpu time, memory,
+ * or other resource limits. Exceeding any specified limit will cause {@link #shouldExit()} to
+ * return true the next time it is checked (it may be checked in either Lucene code or Solr code)
+ */
+public class QueryLimits implements QueryTimeout {
+  private final List<QueryTimeout> limits =
+      new ArrayList<>(3); // timeAllowed, cpu, and memory anticipated
+
+  public static QueryLimits NONE = new QueryLimits();
+
+  private QueryLimits() {}
+
+  /**
+   * Implementors of a Query Limit should add an if block here to activate it, and typically this if
+   * statement will hinge on hasXXXLimit() static method attached to the implementation class.
+   *
+   * @param req the current SolrQueryRequest.
+   */
+  public QueryLimits(SolrQueryRequest req) {
+    if (hasTimeLimit(req)) {
+      limits.add(new SolrQueryTimeLimit(req));
+    }
+  }
+
+  @Override
+  public boolean shouldExit() {
+    for (QueryTimeout limit : limits) {
+      if (limit.shouldExit()) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Method to diagnose limit exceeded. Note that while this should always list the exceeded limit,
+   * it may also nominate additional limits that have been exceeded since the actual check that
+   * cause the failure. This gap is intentional to avoid overly complicated (and possibly expensive)
+   * tracking code that would have to run within the shouldExit method. This method should only be
+   * used to report a failure since it incurs the cost of rechecking every configured limit and does
+   * not short circuit.
+   *
+   * @return A string describing the state pass/fail state of each limit specified for this request.
+   */
+  public String limitStatusMessage() {
+    StringBuilder sb = new StringBuilder();
+    boolean first = true;
+    for (QueryTimeout limit : limits) {
+      if (first) {
+        first = false;
+        sb.append("Query limits:");
+      }
+      sb.append("[");
+      sb.append(limit.getClass().getSimpleName());
+      sb.append(":");
+      sb.append(limit.shouldExit() ? "LIMIT EXCEEDED" : "within limit");
+      sb.append("]");
+    }
+    if (sb.length() == 0) {
+      return "This request is unlimited.";
+    } else {
+      return sb.toString();
+    }
+  }
+
+  public boolean isTimeoutEnabled() {
+    return !limits.isEmpty();
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
index 9c3a2fbc2f3..2520c9ed01b 100644
--- a/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
+++ b/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
@@ -204,7 +204,10 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     assert reader != null;
     reader = UninvertingReader.wrap(reader, core.getLatestSchema().getUninversionMapper());
     if (useExitableDirectoryReader) { // SOLR-16693 legacy; may be removed.  Probably inefficient.
-      reader = ExitableDirectoryReader.wrap(reader, SolrQueryTimeoutImpl.getInstance());
+      SolrRequestInfo requestInfo = SolrRequestInfo.getRequestInfo();
+      assert requestInfo != null;
+      QueryLimits limits = requestInfo.getLimits();
+      reader = ExitableDirectoryReader.wrap(reader, limits);
     }
     return reader;
   }
@@ -714,8 +717,10 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
   @Override
   protected void search(List<LeafReaderContext> leaves, Weight weight, Collector collector)
       throws IOException {
-    final var queryTimeout = SolrQueryTimeoutImpl.getInstance();
-    if (useExitableDirectoryReader || queryTimeout.isTimeoutEnabled() == false) {
+    SolrRequestInfo requestInfo = SolrRequestInfo.getRequestInfo();
+    if (useExitableDirectoryReader
+        || requestInfo == null
+        || !requestInfo.getLimits().isTimeoutEnabled()) {
       // no timeout.  Pass through to super class
       super.search(leaves, weight, collector);
     } else {
@@ -725,10 +730,11 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
       // So we need to make a new IndexSearcher instead of using "this".
       new IndexSearcher(reader) { // cheap, actually!
         void searchWithTimeout() throws IOException {
-          setTimeout(queryTimeout.makeLocalImpl());
+          setTimeout(requestInfo.getLimits()); // Lucene's method name is less than ideal here...
           super.search(leaves, weight, collector); // FYI protected access
           if (timedOut()) {
-            throw new TimeAllowedExceededFromScorerException("timeAllowed exceeded");
+            throw new LimitExceededFromScorerException(
+                "Limits exceeded! " + requestInfo.getLimits().limitStatusMessage());
           }
         }
       }.searchWithTimeout();
@@ -740,10 +746,10 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
    * Further, from the low level Lucene {@code org.apache.lucene.search.TimeLimitingBulkScorer}.
    * Extending {@code ExitableDirectoryReader.ExitingReaderException} is for legacy reasons.
    */
-  public static class TimeAllowedExceededFromScorerException
+  public static class LimitExceededFromScorerException
       extends ExitableDirectoryReader.ExitingReaderException {
 
-    public TimeAllowedExceededFromScorerException(String msg) {
+    public LimitExceededFromScorerException(String msg) {
       super(msg);
     }
   }
@@ -972,7 +978,8 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable, SolrI
     }
 
     DocSet answer;
-    if (SolrQueryTimeoutImpl.getInstance().isTimeoutEnabled()) {
+    SolrRequestInfo requestInfo = SolrRequestInfo.getRequestInfo();
+    if (requestInfo != null && requestInfo.getLimits().isTimeoutEnabled()) {
       // If there is a possibility of timeout for this query, then don't reserve a computation slot.
       // Further, we can't naively wait for an in progress computation to finish, because if we time
       // out before it does then we won't even have partial results to provide. We could possibly
diff --git a/solr/core/src/java/org/apache/solr/search/SolrQueryTimeLimit.java b/solr/core/src/java/org/apache/solr/search/SolrQueryTimeLimit.java
new file mode 100644
index 00000000000..2080d5a922f
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/search/SolrQueryTimeLimit.java
@@ -0,0 +1,66 @@
+/*
+ * 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.solr.search;
+
+import static java.lang.System.nanoTime;
+
+import java.util.concurrent.TimeUnit;
+import org.apache.lucene.index.QueryTimeout;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.request.SolrQueryRequest;
+
+/**
+ * Enforces a wall clock based timeout on a given SolrQueryRequest. This class holds the logic for
+ * the {@code timeAllowed} query parameter. Note that timeAllowed will be ignored for
+ * <strong><em>local</em></strong> processing of sub-queries in cases where the parent query already
+ * has {@code timeAllowed} set. Essentially only one timeAllowed can be specified for any thread
+ * executing a query. This is to ensure that subqueies don't escape from the intended limit
+ */
+public class SolrQueryTimeLimit implements QueryTimeout {
+
+  private final long timeoutAt;
+
+  /**
+   * Create an object to represent a time limit for the current request.
+   *
+   * @param req A solr request that has a value for {@code timeAllowed}
+   * @throws IllegalArgumentException if the request does not contain timeAllowed parameter. This
+   *     should be validated with {@link #hasTimeLimit(SolrQueryRequest)} prior to constructing this
+   *     object
+   */
+  public SolrQueryTimeLimit(SolrQueryRequest req) {
+    // reduce by time already spent
+    long reqTimeAllowed = req.getParams().getLong(CommonParams.TIME_ALLOWED, -1L);
+
+    if (reqTimeAllowed == -1L) {
+      throw new IllegalArgumentException(
+          "Check for limit with hasTimeLimit(req) before creating a SolrQueryTimeLimit");
+    }
+    long timeAllowed = reqTimeAllowed - (long) req.getRequestTimer().getTime();
+    long nanosAllowed = TimeUnit.NANOSECONDS.convert(timeAllowed, TimeUnit.MILLISECONDS);
+    timeoutAt = nanoTime() + nanosAllowed;
+  }
+
+  static boolean hasTimeLimit(SolrQueryRequest req) {
+    return req.getParams().getLong(CommonParams.TIME_ALLOWED, -1L) >= 0L;
+  }
+
+  @Override
+  public boolean shouldExit() {
+    return timeoutAt - nanoTime() < 0L;
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/search/SolrQueryTimeoutImpl.java b/solr/core/src/java/org/apache/solr/search/SolrQueryTimeoutImpl.java
deleted file mode 100644
index a30bb8d0f6c..00000000000
--- a/solr/core/src/java/org/apache/solr/search/SolrQueryTimeoutImpl.java
+++ /dev/null
@@ -1,107 +0,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.
- */
-package org.apache.solr.search;
-
-import static java.lang.System.nanoTime;
-
-import java.util.concurrent.TimeUnit;
-import org.apache.lucene.index.QueryTimeout;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.request.SolrQueryRequest;
-
-/**
- * Implementation of {@link QueryTimeout} that is used by Solr. It uses a ThreadLocal variable to
- * track the timeoutAt value for each request thread.
- */
-public class SolrQueryTimeoutImpl implements QueryTimeout {
-  /** The ThreadLocal variable to store the time beyond which, the processing should exit. */
-  private static final ThreadLocal<Long> timeoutAt = new ThreadLocal<>();
-
-  private static final SolrQueryTimeoutImpl instance = new SolrQueryTimeoutImpl();
-
-  private SolrQueryTimeoutImpl() {}
-
-  /** Return singleton instance */
-  public static SolrQueryTimeoutImpl getInstance() {
-    return instance;
-  }
-
-  /** The time (nanoseconds) at which the request should be considered timed out. */
-  public static Long getTimeoutAtNs() {
-    return timeoutAt.get();
-  }
-
-  public boolean isTimeoutEnabled() {
-    return getTimeoutAtNs() != null;
-  }
-
-  /** Return true if a timeoutAt value is set and the current time has exceeded the set timeOut. */
-  @Override
-  public boolean shouldExit() {
-    Long timeoutAt = getTimeoutAtNs();
-    if (timeoutAt == null) {
-      // timeout unset
-      return false;
-    }
-    return timeoutAt - nanoTime() < 0L;
-  }
-
-  /**
-   * Sets or clears the time allowed based on how much time remains from the start of the request
-   * plus the configured {@link CommonParams#TIME_ALLOWED}.
-   */
-  public static void set(SolrQueryRequest req) {
-    long timeAllowed = req.getParams().getLong(CommonParams.TIME_ALLOWED, -1L);
-    if (timeAllowed >= 0L) {
-      set(timeAllowed - (long) req.getRequestTimer().getTime()); // reduce by time already spent
-    } else {
-      reset();
-    }
-  }
-
-  /**
-   * Sets the time allowed (milliseconds), assuming we start a timer immediately. You should
-   * probably invoke {@link #set(SolrQueryRequest)} instead.
-   */
-  public static void set(long timeAllowed) {
-    long time = nanoTime() + TimeUnit.NANOSECONDS.convert(timeAllowed, TimeUnit.MILLISECONDS);
-    timeoutAt.set(time);
-  }
-
-  /** Cleanup the ThreadLocal timeout value. */
-  public static void reset() {
-    timeoutAt.remove();
-  }
-
-  @Override
-  public String toString() {
-    return "timeoutAt: " + getTimeoutAtNs() + " (System.nanoTime(): " + nanoTime() + ")";
-  }
-
-  /** Internal impl for speed only used when we know there's a timeout enabled. */
-  QueryTimeout makeLocalImpl() {
-    assert isTimeoutEnabled();
-    return new QueryTimeout() {
-      final long timeoutAt = getTimeoutAtNs();
-
-      @Override
-      public boolean shouldExit() {
-        return timeoutAt - nanoTime() < 0L;
-      }
-    };
-  }
-}
diff --git a/solr/core/src/test/org/apache/solr/request/TestSolrRequestInfo.java b/solr/core/src/test/org/apache/solr/request/TestSolrRequestInfo.java
index 088fdf9f345..0864a67d43f 100644
--- a/solr/core/src/test/org/apache/solr/request/TestSolrRequestInfo.java
+++ b/solr/core/src/test/org/apache/solr/request/TestSolrRequestInfo.java
@@ -16,6 +16,9 @@
  */
 package org.apache.solr.request;
 
+import static org.apache.solr.request.SolrRequestInfo.LIMITS_KEY;
+
+import java.util.Objects;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -23,6 +26,7 @@ import java.util.concurrent.atomic.AtomicInteger;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.search.QueryLimits;
 import org.junit.BeforeClass;
 
 public class TestSolrRequestInfo extends SolrTestCaseJ4 {
@@ -78,4 +82,53 @@ public class TestSolrRequestInfo extends SolrTestCaseJ4 {
     assertEquals("hook should be closed only once", 1, counter.get());
     assertNull(SolrRequestInfo.getRequestInfo());
   }
+
+  /**
+   * This test verifies that if the original request has a timeout pushing another SolrRequestInfo
+   * onto the stack will not allow a sub-request that is unlimited (or has a different limit)
+   */
+  public void testLimitsMaintained() {
+    try {
+      LocalSolrQueryRequest timeAllowed1000 =
+          new LocalSolrQueryRequest(h.getCore(), params("timeAllowed", "1000"));
+      LocalSolrQueryRequest timeAllowed20000 =
+          new LocalSolrQueryRequest(h.getCore(), params("timeAllowed", "20000"));
+
+      assertNull(timeAllowed1000.getContext().get(LIMITS_KEY));
+      assertNull(timeAllowed20000.getContext().get(LIMITS_KEY));
+
+      final SolrRequestInfo info1k = new SolrRequestInfo(timeAllowed1000, new SolrQueryResponse());
+      final SolrRequestInfo info20k =
+          new SolrRequestInfo(timeAllowed20000, new SolrQueryResponse());
+
+      // request not modified yet
+      Object limitFrom1k = timeAllowed1000.getContext().get(LIMITS_KEY);
+      assertNull(limitFrom1k);
+      Object limitFrom20k = timeAllowed20000.getContext().get(LIMITS_KEY);
+      assertNull(limitFrom20k);
+
+      SolrRequestInfo.setRequestInfo(info1k);
+      SolrRequestInfo solrRequestInfo = Objects.requireNonNull(SolrRequestInfo.getRequestInfo());
+      assertEquals(solrRequestInfo, info1k);
+
+      QueryLimits limits1k = solrRequestInfo.getLimits();
+      QueryLimits limitsFromReq = (QueryLimits) timeAllowed1000.getContext().get(LIMITS_KEY);
+      assertEquals(limitsFromReq, limits1k);
+
+      SolrRequestInfo.setRequestInfo(info20k); // sub-request
+
+      solrRequestInfo = Objects.requireNonNull(SolrRequestInfo.getRequestInfo());
+      assertEquals(solrRequestInfo, info20k); // pushed onto stack successfully
+
+      // Now verify that the sub-request inherited the limit from the parent
+      limitsFromReq = (QueryLimits) timeAllowed20000.getContext().get(LIMITS_KEY);
+      assertEquals(limitsFromReq, limits1k);
+      QueryLimits limitsFromSRI = SolrRequestInfo.getRequestInfo().getLimits();
+      assertEquals(limitsFromSRI, limits1k);
+    } finally {
+      SolrRequestInfo.clearRequestInfo();
+      SolrRequestInfo.clearRequestInfo();
+      SolrRequestInfo.reset();
+    }
+  }
 }