madrob commented on a change in pull request #1686:
URL: https://github.com/apache/lucene-solr/pull/1686#discussion_r460296287



##########
File path: solr/core/src/java/org/apache/solr/servlet/RequestRateLimiter.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.servlet;
+
+import javax.servlet.AsyncContext;
+import javax.servlet.AsyncEvent;
+import javax.servlet.AsyncListener;
+import javax.servlet.FilterConfig;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Handles rate limiting for a specific request type.
+ *
+ * The control flow is as follows:
+ * Handle request -- Check if slot is available -- If available, acquire slot 
and proceed --
+ * else asynchronously queue the request.
+ *
+ * When an active request completes, a check is performed to see if there are 
any pending requests.
+ * If there is an available pending request, process the same.
+ */
+public class RequestRateLimiter {
+  private static final Logger log = 
LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private Semaphore allowedConcurrentRequests;
+  private RateLimiterConfig rateLimiterConfig;
+  private Queue<AsyncContext> waitQueue;
+  private Queue<AsyncListener> listenerQueue;
+
+  public RequestRateLimiter(RateLimiterConfig rateLimiterConfig) {
+    this.rateLimiterConfig = rateLimiterConfig;
+    this.allowedConcurrentRequests = new 
Semaphore(rateLimiterConfig.allowedRequests);
+    this.waitQueue = new ConcurrentLinkedQueue<>();
+    this.listenerQueue = new ConcurrentLinkedQueue<>();

Review comment:
       We never poll this queue, what is it used for?

##########
File path: solr/core/src/java/org/apache/solr/servlet/RateLimitManager.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.solr.servlet;
+
+import javax.servlet.FilterConfig;
+import javax.servlet.http.HttpServletRequest;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrRequest;
+
+import static 
org.apache.solr.common.params.CommonParams.SOLR_REQUEST_CONTEXT_PARAM;
+import static 
org.apache.solr.common.params.CommonParams.SOLR_REQUEST_TYPE_PARAM;
+
+/**
+ * This class is responsible for managing rate limiting per request type. Rate 
limiters
+ * can be registered with this class against a corresponding type. There can 
be only one
+ * rate limiter associated with a request type.
+ *
+ * The actual rate limiting and the limits should be implemented in the 
corresponding RequestRateLimiter
+ * implementation. RateLimitManager is responsible for the orchestration but 
not the specifics of how the
+ * rate limiting is being done for a specific request type.
+ */
+public class RateLimitManager {
+  public final static int DEFAULT_CONCURRENT_REQUESTS= 10;
+  public final static long DEFAULT_EXPIRATION_TIME_INMS = 300;
+  public final static long DEFAULT_SLOT_ACQUISITION_TIMEOUT_MS = -1;
+
+  private final Map<String, RequestRateLimiter> requestRateLimiterMap;
+
+  public RateLimitManager() {
+    this.requestRateLimiterMap = new HashMap<String, RequestRateLimiter>();

Review comment:
       nit: use `<>` instead of explicit types.

##########
File path: solr/core/src/java/org/apache/solr/servlet/RequestRateLimiter.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.servlet;
+
+import javax.servlet.AsyncContext;
+import javax.servlet.AsyncEvent;
+import javax.servlet.AsyncListener;
+import javax.servlet.FilterConfig;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Handles rate limiting for a specific request type.
+ *
+ * The control flow is as follows:
+ * Handle request -- Check if slot is available -- If available, acquire slot 
and proceed --
+ * else asynchronously queue the request.
+ *
+ * When an active request completes, a check is performed to see if there are 
any pending requests.
+ * If there is an available pending request, process the same.
+ */
+public class RequestRateLimiter {
+  private static final Logger log = 
LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private Semaphore allowedConcurrentRequests;
+  private RateLimiterConfig rateLimiterConfig;
+  private Queue<AsyncContext> waitQueue;
+  private Queue<AsyncListener> listenerQueue;
+
+  public RequestRateLimiter(RateLimiterConfig rateLimiterConfig) {
+    this.rateLimiterConfig = rateLimiterConfig;
+    this.allowedConcurrentRequests = new 
Semaphore(rateLimiterConfig.allowedRequests);
+    this.waitQueue = new ConcurrentLinkedQueue<>();
+    this.listenerQueue = new ConcurrentLinkedQueue<>();
+  }
+
+  public boolean handleRequest(HttpServletRequest request) throws 
InterruptedException {
+
+    if (!rateLimiterConfig.isEnabled) {
+      return true;
+    }
+
+    boolean accepted = 
allowedConcurrentRequests.tryAcquire(rateLimiterConfig.waitForSlotAcquisition, 
TimeUnit.MILLISECONDS);
+
+    if (!accepted) {
+      AsyncContext asyncContext = request.startAsync();
+      AsyncListener asyncListener = buildAsyncListener();
+
+      if (rateLimiterConfig.requestExpirationTimeInMS > 0) {
+        asyncContext.setTimeout(rateLimiterConfig.requestExpirationTimeInMS);
+      }
+
+      asyncContext.addListener(asyncListener);
+      listenerQueue.add(asyncListener);
+      waitQueue.add(asyncContext);
+    }
+
+    return accepted;
+  }
+
+  public boolean resumePendingOperation() {
+    AsyncContext asyncContext = waitQueue.poll();
+
+    if (asyncContext != null) {
+      try {
+        asyncContext.dispatch();
+        return true;
+      }
+      catch (IllegalStateException x) {
+        if (log.isWarnEnabled()) {
+          String errorMessage = x.getMessage();
+          log.warn(errorMessage);
+        }
+      }
+    }
+
+    return false;
+  }
+
+  public void decrementConcurrentRequests() {
+    allowedConcurrentRequests.release();
+  }
+
+  public void close() {
+    while (!waitQueue.isEmpty()) {
+      AsyncContext asyncContext = waitQueue.poll();
+
+      asyncContext.complete();
+    }
+
+    listenerQueue.clear();
+  }
+
+  private AsyncListener buildAsyncListener() {
+    return new AsyncListener() {
+      @Override
+      public void onComplete(AsyncEvent asyncEvent) throws IOException {
+
+      }
+
+      @Override
+      public void onTimeout(AsyncEvent asyncEvent) throws IOException {
+        AsyncContext asyncContext = asyncEvent.getAsyncContext();
+
+        if (!waitQueue.remove(asyncContext)) {
+          return;
+        }
+
+        HttpServletResponse servletResponse = 
((HttpServletResponse)asyncEvent.getSuppliedResponse());
+        String responseMessage = "Too many requests for this request type." +

Review comment:
       Unused.

##########
File path: solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
##########
@@ -377,6 +389,16 @@ public void doFilter(ServletRequest _request, 
ServletResponse _response, FilterC
         }
       }
 
+      try {
+        accepted = rateLimitManager.handleRequest(request);
+      } catch (InterruptedException e) {
+        throw new RuntimeException(e.getMessage());

Review comment:
       SolrException or SolrServerException probably

##########
File path: solr/core/src/java/org/apache/solr/servlet/RateLimitManager.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.solr.servlet;
+
+import javax.servlet.FilterConfig;
+import javax.servlet.http.HttpServletRequest;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrRequest;
+
+import static 
org.apache.solr.common.params.CommonParams.SOLR_REQUEST_CONTEXT_PARAM;
+import static 
org.apache.solr.common.params.CommonParams.SOLR_REQUEST_TYPE_PARAM;
+
+/**
+ * This class is responsible for managing rate limiting per request type. Rate 
limiters
+ * can be registered with this class against a corresponding type. There can 
be only one
+ * rate limiter associated with a request type.
+ *
+ * The actual rate limiting and the limits should be implemented in the 
corresponding RequestRateLimiter
+ * implementation. RateLimitManager is responsible for the orchestration but 
not the specifics of how the
+ * rate limiting is being done for a specific request type.
+ */
+public class RateLimitManager {
+  public final static int DEFAULT_CONCURRENT_REQUESTS= 10;
+  public final static long DEFAULT_EXPIRATION_TIME_INMS = 300;
+  public final static long DEFAULT_SLOT_ACQUISITION_TIMEOUT_MS = -1;
+
+  private final Map<String, RequestRateLimiter> requestRateLimiterMap;
+
+  public RateLimitManager() {
+    this.requestRateLimiterMap = new HashMap<String, RequestRateLimiter>();
+  }
+
+  // Handles an incoming request. The main orchestration code path, this 
method will
+  // identify which (if any) rate limiter can handle this request. Internal 
requests will not be
+  // rate limited
+  // Returns true if request is accepted for processing, false if it should be 
rejected
+
+  // NOTE: It is upto specific rate limiter implementation to handle queuing 
of rejected requests.
+  public boolean handleRequest(HttpServletRequest request) throws 
InterruptedException {
+    String requestContext = request.getHeader(SOLR_REQUEST_CONTEXT_PARAM);
+    String typeOfRequest = request.getHeader(SOLR_REQUEST_TYPE_PARAM);
+
+    if (typeOfRequest == null) {
+      // Cannot determine if this request should be throttled
+      return true;
+    }
+
+    // Do not throttle internal requests
+    if (requestContext != null && 
requestContext.equals(SolrRequest.SolrClientContext.SERVER.toString())) {
+      return true;
+    }
+
+    RequestRateLimiter requestRateLimiter = 
requestRateLimiterMap.get(typeOfRequest);
+
+    if (requestRateLimiter == null) {
+      // No request rate limiter for this request type
+      return true;
+    }
+
+    return requestRateLimiter.handleRequest(request);
+  }
+
+  // Resume a pending request from one of the registered rate limiters.
+  // The current model is round robin -- iterate over the list and get a 
pending request and resume it.
+
+  // TODO: This should be a priority queue based model
+  public void resumePendingRequest(HttpServletRequest request) {
+    String typeOfRequest = request.getHeader(SOLR_REQUEST_TYPE_PARAM);
+
+    RequestRateLimiter previousRequestRateLimiter = 
requestRateLimiterMap.get(typeOfRequest);
+
+    if (previousRequestRateLimiter == null) {
+      // No rate limiter for this request type
+      return;
+    }
+
+    // Give preference to the previous request's rate limiter
+    if (previousRequestRateLimiter.resumePendingOperation()) {
+      return;
+    }
+
+    for (Map.Entry<String, RequestRateLimiter> currentEntry : 
requestRateLimiterMap.entrySet()) {

Review comment:
       I'm confused as to why this exists. If a query request completes, why 
does that mean that we have room for an update request now?

##########
File path: solr/solr-ref-guide/src/rate-limiters.adoc
##########
@@ -0,0 +1,101 @@
+= Request Rate Limiters
+// 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.
+
+Solr allows rate limiting per request type. Each request type can be allocated 
a maximum allowed number of concurrent requests
+that can be active. The default rate limiting is implemented for updates and 
searches.
+
+If a request exceeds the request quota, further incoming requests are 
automatically queued asynchronously with
+a configurable timeout.
+
+== When To Use Rate Limiters
+Rate limiters should be used when the user wishes to allocate a guaranteed 
capacity of the request threadpool to a specific
+request type. Indexing and search requests are mostly competing with each 
other for CPU resources. This becomes especially
+pronounced under high stress in production workloads.
+
+== Rate Limiter Configurations
+The default rate limiter is search rate limiter. Accordingly, it can be 
configured in web.xml under initParams for

Review comment:
       Is there anywhere else that we can configure this? I thought the trend 
was to move away from putting things in `web.xml` as much as possible.

##########
File path: solr/solr-ref-guide/src/rate-limiters.adoc
##########
@@ -0,0 +1,101 @@
+= Request Rate Limiters
+// 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.
+
+Solr allows rate limiting per request type. Each request type can be allocated 
a maximum allowed number of concurrent requests
+that can be active. The default rate limiting is implemented for updates and 
searches.
+
+If a request exceeds the request quota, further incoming requests are 
automatically queued asynchronously with
+a configurable timeout.
+
+== When To Use Rate Limiters
+Rate limiters should be used when the user wishes to allocate a guaranteed 
capacity of the request threadpool to a specific
+request type. Indexing and search requests are mostly competing with each 
other for CPU resources. This becomes especially
+pronounced under high stress in production workloads.
+
+== Rate Limiter Configurations
+The default rate limiter is search rate limiter. Accordingly, it can be 
configured in web.xml under initParams for
+SolrRequestFilter.
+
+[source,xml]
+----
+<filter-name>SolrRequestFilter</filter-name>
+----
+
+==== Enable Query Rate Limiter
+Controls enabling of query rate limiter. Default value is false.
+[source,xml]
+----
+<param-name>isQueryRateLimiterEnabled</param-name>
+----
+[source,xml]
+----
+<param-value>true</param-value>
+----
+
+==== Maximum Number Of Concurrent Requests
+Allows setting maximum concurrent search requests at a given point in time. 
Default value is 10.
+[source,xml]
+----
+<param-name>maxQueryRequests</param-name>
+----
+[source,xml]
+----
+<param-value>15</param-value>
+----
+
+==== Request Slot Allocation Wait Time
+Wait time in ms for which a request will wait for a slot to be available when 
all slots are full,
+before the request is put into the wait queue. This allows requests to have a 
chance to proceed if
+the unavailability of the request slots for this rate limiter is a transient 
phenomenon. Default value
+is -1, indicating no wait.
+[source,xml]
+----
+<param-name>queryWaitForSlotAllocationInMS</param-name>
+----
+[source,xml]
+----
+<param-value>100</param-value>
+----
+
+==== Request Expiration Time
+Time in ms after which a request will expire in the wait queue. Default value 
is 200 ms.

Review comment:
       What does 0 mean here? Do negative values mean no expiration?

##########
File path: solr/solr-ref-guide/src/rate-limiters.adoc
##########
@@ -0,0 +1,101 @@
+= Request Rate Limiters
+// 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.
+
+Solr allows rate limiting per request type. Each request type can be allocated 
a maximum allowed number of concurrent requests
+that can be active. The default rate limiting is implemented for updates and 
searches.
+
+If a request exceeds the request quota, further incoming requests are 
automatically queued asynchronously with
+a configurable timeout.
+
+== When To Use Rate Limiters
+Rate limiters should be used when the user wishes to allocate a guaranteed 
capacity of the request threadpool to a specific
+request type. Indexing and search requests are mostly competing with each 
other for CPU resources. This becomes especially
+pronounced under high stress in production workloads.
+
+== Rate Limiter Configurations
+The default rate limiter is search rate limiter. Accordingly, it can be 
configured in web.xml under initParams for
+SolrRequestFilter.
+
+[source,xml]
+----
+<filter-name>SolrRequestFilter</filter-name>
+----
+
+==== Enable Query Rate Limiter
+Controls enabling of query rate limiter. Default value is false.
+[source,xml]
+----
+<param-name>isQueryRateLimiterEnabled</param-name>
+----
+[source,xml]
+----
+<param-value>true</param-value>
+----
+
+==== Maximum Number Of Concurrent Requests
+Allows setting maximum concurrent search requests at a given point in time. 
Default value is 10.

Review comment:
       This is a maximum number for the JVM? Not collection or core specific? 
10 seems like a very low default.

##########
File path: solr/solr-ref-guide/src/rate-limiters.adoc
##########
@@ -0,0 +1,101 @@
+= Request Rate Limiters
+// 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.
+
+Solr allows rate limiting per request type. Each request type can be allocated 
a maximum allowed number of concurrent requests
+that can be active. The default rate limiting is implemented for updates and 
searches.
+
+If a request exceeds the request quota, further incoming requests are 
automatically queued asynchronously with
+a configurable timeout.
+
+== When To Use Rate Limiters
+Rate limiters should be used when the user wishes to allocate a guaranteed 
capacity of the request threadpool to a specific
+request type. Indexing and search requests are mostly competing with each 
other for CPU resources. This becomes especially
+pronounced under high stress in production workloads.
+
+== Rate Limiter Configurations
+The default rate limiter is search rate limiter. Accordingly, it can be 
configured in web.xml under initParams for
+SolrRequestFilter.
+
+[source,xml]
+----
+<filter-name>SolrRequestFilter</filter-name>
+----
+
+==== Enable Query Rate Limiter
+Controls enabling of query rate limiter. Default value is false.
+[source,xml]
+----
+<param-name>isQueryRateLimiterEnabled</param-name>
+----
+[source,xml]
+----
+<param-value>true</param-value>
+----
+
+==== Maximum Number Of Concurrent Requests
+Allows setting maximum concurrent search requests at a given point in time. 
Default value is 10.
+[source,xml]
+----
+<param-name>maxQueryRequests</param-name>
+----
+[source,xml]
+----
+<param-value>15</param-value>
+----
+
+==== Request Slot Allocation Wait Time
+Wait time in ms for which a request will wait for a slot to be available when 
all slots are full,
+before the request is put into the wait queue. This allows requests to have a 
chance to proceed if
+the unavailability of the request slots for this rate limiter is a transient 
phenomenon. Default value
+is -1, indicating no wait.
+[source,xml]
+----
+<param-name>queryWaitForSlotAllocationInMS</param-name>
+----
+[source,xml]
+----
+<param-value>100</param-value>
+----
+
+==== Request Expiration Time
+Time in ms after which a request will expire in the wait queue. Default value 
is 200 ms.
+[source,xml]
+----
+<param-name>queryRequestExpirationTimeInMS</param-name>
+----
+[source,xml]
+----
+<param-value>200</param-value>
+----
+
+== Salient Points
+
+These are some of the things to keep in mind when using rate limiters
+
+=== Over Subscribing
+It is possible to define a size of quota for a request type which exceeds the 
size
+of the available threadpool. Solr does not enforce rules on the size of a 
quota that
+can be define for a request type. This is intentionally done to allow users 
full
+control on their quota allocation. However, if the quota exceeds the available 
threadpool's
+size, the standard queuing policies of the threadpool will kick in.
+
+=== Work Stealing

Review comment:
       This should be configurable to enable/disable

##########
File path: solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
##########
@@ -441,6 +463,13 @@ public void doFilter(ServletRequest _request, 
ServletResponse _response, FilterC
       consumeInputFully(request, response);
       SolrRequestInfo.reset();
       SolrRequestParsers.cleanupMultipartFiles(request);
+
+      if (accepted) {
+        rateLimitManager.decrementActiveRequests(request);

Review comment:
       There is a race condition here where new incoming requests can get in 
before resume of an existing request.

##########
File path: solr/core/src/java/org/apache/solr/servlet/RequestRateLimiter.java
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.servlet;
+
+import javax.servlet.AsyncContext;
+import javax.servlet.AsyncEvent;
+import javax.servlet.AsyncListener;
+import javax.servlet.FilterConfig;
+import javax.servlet.http.HttpServletRequest;
+import javax.servlet.http.HttpServletResponse;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Handles rate limiting for a specific request type.
+ *
+ * The control flow is as follows:
+ * Handle request -- Check if slot is available -- If available, acquire slot 
and proceed --
+ * else asynchronously queue the request.
+ *
+ * When an active request completes, a check is performed to see if there are 
any pending requests.
+ * If there is an available pending request, process the same.
+ */
+public class RequestRateLimiter {
+  private static final Logger log = 
LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private Semaphore allowedConcurrentRequests;
+  private RateLimiterConfig rateLimiterConfig;
+  private Queue<AsyncContext> waitQueue;
+  private Queue<AsyncListener> listenerQueue;
+
+  public RequestRateLimiter(RateLimiterConfig rateLimiterConfig) {
+    this.rateLimiterConfig = rateLimiterConfig;
+    this.allowedConcurrentRequests = new 
Semaphore(rateLimiterConfig.allowedRequests);
+    this.waitQueue = new ConcurrentLinkedQueue<>();
+    this.listenerQueue = new ConcurrentLinkedQueue<>();
+  }
+
+  public boolean handleRequest(HttpServletRequest request) throws 
InterruptedException {
+
+    if (!rateLimiterConfig.isEnabled) {
+      return true;
+    }
+
+    boolean accepted = 
allowedConcurrentRequests.tryAcquire(rateLimiterConfig.waitForSlotAcquisition, 
TimeUnit.MILLISECONDS);
+
+    if (!accepted) {
+      AsyncContext asyncContext = request.startAsync();

Review comment:
       Is this different from admin requests sent with `async=[async-id]`?

##########
File path: solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
##########
@@ -331,6 +338,10 @@ public void close() {
           metricManager = null;
         }
       }
+
+      if (rateLimitManager != null) {

Review comment:
       should this be in the finally block instead?

##########
File path: 
solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
##########
@@ -358,7 +358,11 @@ protected HttpRequestBase 
createMethod(@SuppressWarnings({"rawtypes"})SolrReques
     if (parser == null) {
       parser = this.parser;
     }
-    
+
+    Header[] contextHeaders = new Header[2];
+    contextHeaders[0] = new 
BasicHeader(CommonParams.SOLR_REQUEST_CONTEXT_PARAM, getContext().toString());

Review comment:
       This seems trivial to forge. Can we tie this into the authentication 
code paths somehow so that we can really trust that the request is coming from 
a server instead of just some application claiming to be a server?

##########
File path: solr/solr-ref-guide/src/rate-limiters.adoc
##########
@@ -0,0 +1,101 @@
+= Request Rate Limiters
+// 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.
+
+Solr allows rate limiting per request type. Each request type can be allocated 
a maximum allowed number of concurrent requests
+that can be active. The default rate limiting is implemented for updates and 
searches.
+
+If a request exceeds the request quota, further incoming requests are 
automatically queued asynchronously with
+a configurable timeout.
+
+== When To Use Rate Limiters
+Rate limiters should be used when the user wishes to allocate a guaranteed 
capacity of the request threadpool to a specific
+request type. Indexing and search requests are mostly competing with each 
other for CPU resources. This becomes especially
+pronounced under high stress in production workloads.
+
+== Rate Limiter Configurations
+The default rate limiter is search rate limiter. Accordingly, it can be 
configured in web.xml under initParams for
+SolrRequestFilter.
+
+[source,xml]
+----
+<filter-name>SolrRequestFilter</filter-name>
+----
+
+==== Enable Query Rate Limiter
+Controls enabling of query rate limiter. Default value is false.
+[source,xml]
+----
+<param-name>isQueryRateLimiterEnabled</param-name>
+----
+[source,xml]
+----
+<param-value>true</param-value>
+----
+
+==== Maximum Number Of Concurrent Requests
+Allows setting maximum concurrent search requests at a given point in time. 
Default value is 10.
+[source,xml]
+----
+<param-name>maxQueryRequests</param-name>
+----
+[source,xml]
+----
+<param-value>15</param-value>
+----
+
+==== Request Slot Allocation Wait Time
+Wait time in ms for which a request will wait for a slot to be available when 
all slots are full,
+before the request is put into the wait queue. This allows requests to have a 
chance to proceed if
+the unavailability of the request slots for this rate limiter is a transient 
phenomenon. Default value
+is -1, indicating no wait.

Review comment:
       What does a value of 0 mean?




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

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



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

Reply via email to