You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2018/05/22 13:57:53 UTC

lucene-solr:branch_7x: SOLR-11880: Avoid creating new exceptions for every request made to MDCAwareThreadPoolExecutor by distributed search and update operations

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x 68bf65c1e -> c4a41c4c6


SOLR-11880: Avoid creating new exceptions for every request made to MDCAwareThreadPoolExecutor by distributed search and update operations

(cherry picked from commit 5a47ed4)


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/c4a41c4c
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/c4a41c4c
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/c4a41c4c

Branch: refs/heads/branch_7x
Commit: c4a41c4c6a368f4461f6d31cfa6647348a3f7666
Parents: 68bf65c
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Tue May 22 19:26:42 2018 +0530
Committer: Shalin Shekhar Mangar <sh...@apache.org>
Committed: Tue May 22 19:27:45 2018 +0530

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  3 +++
 .../component/HttpShardHandlerFactory.java      |  7 +++++--
 .../apache/solr/update/UpdateShardHandler.java  | 11 ++++++++--
 .../apache/solr/common/util/ExecutorUtil.java   | 22 ++++++++++++++++----
 4 files changed, 35 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c4a41c4c/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index aac103a..1d5b285 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -221,6 +221,9 @@ Optimizations
 
 * SOLR-12333: Removed redundant lines for handling lists in JSON reponse writers. (David Smiley via Mikhail Khludnev)
 
+* SOLR-11880: Avoid creating new exceptions for every request made to MDCAwareThreadPoolExecutor by distributed
+  search and update operations. (Varun Thacker, shalin)
+
 Other Changes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c4a41c4c/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
index 4e2a794..b0d3dd8 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
@@ -81,8 +81,11 @@ public class HttpShardHandlerFactory extends ShardHandlerFactory implements org.
       0,
       Integer.MAX_VALUE,
       5, TimeUnit.SECONDS, // terminate idle threads after 5 sec
-      new SynchronousQueue<Runnable>(),  // directly hand off tasks
-      new DefaultSolrThreadFactory("httpShardExecutor")
+      new SynchronousQueue<>(),  // directly hand off tasks
+      new DefaultSolrThreadFactory("httpShardExecutor"),
+      // the Runnable added to this executor handles all exceptions so we disable stack trace collection as an optimization
+      // see SOLR-11880 for more details
+      false
   );
 
   protected InstrumentedPoolingHttpClientConnectionManager clientConnectionManager;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c4a41c4c/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java b/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
index ecc843f..8adf902 100644
--- a/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
+++ b/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
@@ -20,7 +20,9 @@ import java.lang.invoke.MethodHandles;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ExecutorService;
+import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
 
 import com.codahale.metrics.MetricRegistry;
 import org.apache.http.client.HttpClient;
@@ -52,8 +54,13 @@ public class UpdateShardHandler implements SolrMetricProducer, SolrInfoBean {
    * and then undetected shard inconsistency as a result.
    * Therefore this thread pool is left unbounded. See SOLR-8205
    */
-  private ExecutorService updateExecutor = ExecutorUtil.newMDCAwareCachedThreadPool(
-      new SolrjNamedThreadFactory("updateExecutor"));
+  private ExecutorService updateExecutor = new ExecutorUtil.MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
+      60L, TimeUnit.SECONDS,
+      new SynchronousQueue<>(),
+      new SolrjNamedThreadFactory("updateExecutor"),
+      // the Runnable added to this executor handles all exceptions so we disable stack trace collection as an optimization
+      // see SOLR-11880 for more details
+      false);
   
   private ExecutorService recoveryExecutor;
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/c4a41c4c/solr/solrj/src/java/org/apache/solr/common/util/ExecutorUtil.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/ExecutorUtil.java b/solr/solrj/src/java/org/apache/solr/common/util/ExecutorUtil.java
index 7458016..f56f782 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/ExecutorUtil.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/ExecutorUtil.java
@@ -105,7 +105,7 @@ public class ExecutorUtil {
   public static ExecutorService newMDCAwareSingleThreadExecutor(ThreadFactory threadFactory) {
     return new MDCAwareThreadPoolExecutor(1, 1,
             0L, TimeUnit.MILLISECONDS,
-            new LinkedBlockingQueue<Runnable>(),
+            new LinkedBlockingQueue<>(),
             threadFactory);
   }
 
@@ -122,7 +122,7 @@ public class ExecutorUtil {
   public static ExecutorService newMDCAwareCachedThreadPool(ThreadFactory threadFactory) {
     return new MDCAwareThreadPoolExecutor(0, Integer.MAX_VALUE,
         60L, TimeUnit.SECONDS,
-        new SynchronousQueue<Runnable>(),
+        new SynchronousQueue<>(),
         threadFactory);
   }
 
@@ -131,20 +131,30 @@ public class ExecutorUtil {
 
     private static final int MAX_THREAD_NAME_LEN = 512;
 
+    private final boolean enableSubmitterStackTrace;
+
     public MDCAwareThreadPoolExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime, TimeUnit unit, BlockingQueue<Runnable> workQueue, ThreadFactory threadFactory, RejectedExecutionHandler handler) {
       super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, handler);
+      this.enableSubmitterStackTrace = true;
     }
 
     public MDCAwareThreadPoolExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime, TimeUnit unit, BlockingQueue<Runnable> workQueue) {
       super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue);
+      this.enableSubmitterStackTrace = true;
     }
 
     public MDCAwareThreadPoolExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime, TimeUnit unit, BlockingQueue<Runnable> workQueue, ThreadFactory threadFactory) {
+      this(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory, true);
+    }
+
+    public MDCAwareThreadPoolExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime, TimeUnit unit, BlockingQueue<Runnable> workQueue, ThreadFactory threadFactory, boolean enableSubmitterStackTrace) {
       super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, threadFactory);
+      this.enableSubmitterStackTrace = enableSubmitterStackTrace;
     }
 
     public MDCAwareThreadPoolExecutor(int corePoolSize, int maximumPoolSize, long keepAliveTime, TimeUnit unit, BlockingQueue<Runnable> workQueue, RejectedExecutionHandler handler) {
       super(corePoolSize, maximumPoolSize, keepAliveTime, unit, workQueue, handler);
+      this.enableSubmitterStackTrace = true;
     }
 
     @Override
@@ -164,7 +174,7 @@ public class ExecutorUtil {
 
       String ctxStr = contextString.toString().replace("/", "//");
       final String submitterContextStr = ctxStr.length() <= MAX_THREAD_NAME_LEN ? ctxStr : ctxStr.substring(0, MAX_THREAD_NAME_LEN);
-      final Exception submitterStackTrace = new Exception("Submitter stack trace");
+      final Exception submitterStackTrace = enableSubmitterStackTrace ? new Exception("Submitter stack trace") : null;
       final List<InheritableThreadLocalProvider> providersCopy = providers;
       final ArrayList<AtomicReference> ctx = providersCopy.isEmpty() ? null : new ArrayList<>(providersCopy.size());
       if (ctx != null) {
@@ -194,7 +204,11 @@ public class ExecutorUtil {
           if (t instanceof OutOfMemoryError) {
             throw t;
           }
-          log.error("Uncaught exception {} thrown by thread: {}", t, currentThread.getName(), submitterStackTrace);
+          if (enableSubmitterStackTrace)  {
+            log.error("Uncaught exception {} thrown by thread: {}", t, currentThread.getName(), submitterStackTrace);
+          } else  {
+            log.error("Uncaught exception {} thrown by thread: {}", t, currentThread.getName());
+          }
           throw t;
         } finally {
           isServerPool.remove();