You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sa...@apache.org on 2016/06/17 21:44:35 UTC

[1/2] lucene-solr:branch_5_5: SOLR-9116: Race condition causing occasional SolrIndexSearcher leak when SolrCore is reloaded (cherry picked from commit d6a5c5a)

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_5_5 8fcaa8568 -> e86eb9d1e
  refs/heads/branch_5x 1df6d4f07 -> d58bff079


SOLR-9116: Race condition causing occasional SolrIndexSearcher leak when SolrCore is reloaded
(cherry picked from commit d6a5c5a)


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

Branch: refs/heads/branch_5_5
Commit: e86eb9d1ecd6f63b15068b368819e31c4c3e1203
Parents: 8fcaa85
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Wed May 18 20:15:52 2016 +0530
Committer: Steve Rowe <sa...@apache.org>
Committed: Fri Jun 17 17:43:35 2016 -0400

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  3 +
 .../src/java/org/apache/solr/core/SolrCore.java | 20 +++++++
 .../test/org/apache/solr/core/SolrCoreTest.java | 61 +++++++++++++++++++-
 3 files changed, 83 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e86eb9d1/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index a671684..75028ce 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -68,6 +68,9 @@ Bug Fixes
 
 * SOLR-9117: The first SolrCore is leaked after reload. (Jessica Cheng Mallet via shalin)
 
+* SOLR-9116: Race condition causing occasional SolrIndexSearcher leak when SolrCore is reloaded.
+  (Jessica Cheng Mallet via shalin)
+
 ======================= 5.5.1 =======================
 
 Bug Fixes

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e86eb9d1/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index 8f92ed6..458341a 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -1464,6 +1464,18 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   private RefCounted<SolrIndexSearcher> realtimeSearcher;
   private Callable<DirectoryReader> newReaderCreator;
 
+  // For testing
+  boolean areAllSearcherReferencesEmpty() {
+    boolean isEmpty;
+    synchronized (searcherLock) {
+      isEmpty = _searchers.isEmpty();
+      isEmpty = isEmpty && _realtimeSearchers.isEmpty();
+      isEmpty = isEmpty && (_searcher == null);
+      isEmpty = isEmpty && (realtimeSearcher == null);
+    }
+    return isEmpty;
+  }
+
   /**
   * Return a registered {@link RefCounted}&lt;{@link SolrIndexSearcher}&gt; with
   * the reference count incremented.  It <b>must</b> be decremented when no longer needed.
@@ -1663,6 +1675,14 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
       newSearcher.incref();
 
       synchronized (searcherLock) {
+        // Check if the core is closed again inside the lock in case this method is racing with a close. If the core is
+        // closed, clean up the new searcher and bail.
+        if (isClosed()) {
+          newSearcher.decref(); // once for caller since we're not returning it
+          newSearcher.decref(); // once for ourselves since it won't be "replaced"
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "openNewSearcher called on closed core");
+        }
+
         if (realtimeSearcher != null) {
           realtimeSearcher.decref();
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e86eb9d1/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java b/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
index 5482707..4ae8af0 100644
--- a/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
+++ b/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
@@ -17,6 +17,7 @@
 package org.apache.solr.core;
 
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.handler.component.QueryComponent;
@@ -24,7 +25,9 @@ import org.apache.solr.handler.component.SpellCheckComponent;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.util.DefaultSolrThreadFactory;
+import org.apache.solr.util.RefCounted;
 import org.apache.solr.util.plugin.SolrCoreAware;
 import org.junit.Test;
 
@@ -33,7 +36,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
@@ -217,6 +219,63 @@ public class SolrCoreTest extends SolrTestCaseJ4 {
     assertEquals("wrong config for queryResultWindowSize", 10, solrConfig.queryResultWindowSize);
   }
 
+  /**
+   * Test that's meant to be run with many iterations to expose a leak of SolrIndexSearcher when a core is closed
+   * due to a reload. Without the fix, this test fails with most iters=1000 runs.
+   */
+  @Test
+  public void testReloadLeak() throws Exception {
+    final ExecutorService executor =
+        ExecutorUtil.newMDCAwareFixedThreadPool(1, new DefaultSolrThreadFactory("testReloadLeak"));
+
+    // Continuously open new searcher while core is not closed, and reload core to try to reproduce searcher leak.
+    // While in practice we never continuously open new searchers, this is trying to make up for the fact that opening
+    // a searcher in this empty core is very fast by opening new searchers continuously to increase the likelihood
+    // for race.
+    SolrCore core = h.getCore();
+    assertTrue("Refcount != 1", core.getOpenCount() == 1);
+    executor.execute(new NewSearcherRunnable(core));
+
+    // Since we called getCore() vs getCoreInc() and don't own a refCount, the container should decRef the core
+    // and close it when we call reload.
+    h.reload();
+
+    executor.shutdown();
+    executor.awaitTermination(1, TimeUnit.MINUTES);
+
+    // Check that all cores are closed and no searcher references are leaked.
+    assertTrue("SolrCore " + core + " is not closed", core.isClosed());
+    assertTrue(core.areAllSearcherReferencesEmpty());
+  }
+
+  private static class NewSearcherRunnable implements Runnable {
+    private final SolrCore core;
+
+    NewSearcherRunnable(SolrCore core) {
+      this.core = core;
+    }
+
+    @Override
+    public void run() {
+      while (!core.isClosed()) {
+        try {
+          RefCounted<SolrIndexSearcher> newSearcher = null;
+          try {
+            newSearcher = core.openNewSearcher(true, true);
+          } finally {
+            if (newSearcher != null) {
+              newSearcher.decref();
+            }
+          }
+        } catch (SolrException e) {
+          if (!core.isClosed()) {
+            throw e;
+          }
+        }
+      }
+    }
+  }
+
 }
 
 


[2/2] lucene-solr:branch_5x: SOLR-9116: Race condition causing occasional SolrIndexSearcher leak when SolrCore is reloaded (cherry picked from commit d6a5c5a)

Posted by sa...@apache.org.
SOLR-9116: Race condition causing occasional SolrIndexSearcher leak when SolrCore is reloaded
(cherry picked from commit d6a5c5a)


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

Branch: refs/heads/branch_5x
Commit: d58bff079cca06fa7d54ced6d73fa7ffb8e0df7f
Parents: 1df6d4f
Author: Shalin Shekhar Mangar <sh...@apache.org>
Authored: Wed May 18 20:15:52 2016 +0530
Committer: Steve Rowe <sa...@apache.org>
Committed: Fri Jun 17 17:44:19 2016 -0400

----------------------------------------------------------------------
 .../src/java/org/apache/solr/core/SolrCore.java | 20 +++++++
 .../test/org/apache/solr/core/SolrCoreTest.java | 61 +++++++++++++++++++-
 2 files changed, 80 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d58bff07/solr/core/src/java/org/apache/solr/core/SolrCore.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/core/SolrCore.java b/solr/core/src/java/org/apache/solr/core/SolrCore.java
index 8f92ed6..458341a 100644
--- a/solr/core/src/java/org/apache/solr/core/SolrCore.java
+++ b/solr/core/src/java/org/apache/solr/core/SolrCore.java
@@ -1464,6 +1464,18 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
   private RefCounted<SolrIndexSearcher> realtimeSearcher;
   private Callable<DirectoryReader> newReaderCreator;
 
+  // For testing
+  boolean areAllSearcherReferencesEmpty() {
+    boolean isEmpty;
+    synchronized (searcherLock) {
+      isEmpty = _searchers.isEmpty();
+      isEmpty = isEmpty && _realtimeSearchers.isEmpty();
+      isEmpty = isEmpty && (_searcher == null);
+      isEmpty = isEmpty && (realtimeSearcher == null);
+    }
+    return isEmpty;
+  }
+
   /**
   * Return a registered {@link RefCounted}&lt;{@link SolrIndexSearcher}&gt; with
   * the reference count incremented.  It <b>must</b> be decremented when no longer needed.
@@ -1663,6 +1675,14 @@ public final class SolrCore implements SolrInfoMBean, Closeable {
       newSearcher.incref();
 
       synchronized (searcherLock) {
+        // Check if the core is closed again inside the lock in case this method is racing with a close. If the core is
+        // closed, clean up the new searcher and bail.
+        if (isClosed()) {
+          newSearcher.decref(); // once for caller since we're not returning it
+          newSearcher.decref(); // once for ourselves since it won't be "replaced"
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "openNewSearcher called on closed core");
+        }
+
         if (realtimeSearcher != null) {
           realtimeSearcher.decref();
         }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/d58bff07/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java b/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
index 88ade19..3248fd9 100644
--- a/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
+++ b/solr/core/src/test/org/apache/solr/core/SolrCoreTest.java
@@ -17,6 +17,7 @@
 package org.apache.solr.core;
 
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.ExecutorUtil;
 import org.apache.solr.handler.ReplicationHandler;
 import org.apache.solr.handler.RequestHandlerBase;
@@ -25,7 +26,9 @@ import org.apache.solr.handler.component.SpellCheckComponent;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestHandler;
 import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.util.DefaultSolrThreadFactory;
+import org.apache.solr.util.RefCounted;
 import org.apache.solr.util.plugin.SolrCoreAware;
 import org.junit.Test;
 
@@ -35,7 +38,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 
@@ -255,6 +257,63 @@ public class SolrCoreTest extends SolrTestCaseJ4 {
     assertEquals("wrong config for queryResultWindowSize", 10, solrConfig.queryResultWindowSize);
   }
 
+  /**
+   * Test that's meant to be run with many iterations to expose a leak of SolrIndexSearcher when a core is closed
+   * due to a reload. Without the fix, this test fails with most iters=1000 runs.
+   */
+  @Test
+  public void testReloadLeak() throws Exception {
+    final ExecutorService executor =
+        ExecutorUtil.newMDCAwareFixedThreadPool(1, new DefaultSolrThreadFactory("testReloadLeak"));
+
+    // Continuously open new searcher while core is not closed, and reload core to try to reproduce searcher leak.
+    // While in practice we never continuously open new searchers, this is trying to make up for the fact that opening
+    // a searcher in this empty core is very fast by opening new searchers continuously to increase the likelihood
+    // for race.
+    SolrCore core = h.getCore();
+    assertTrue("Refcount != 1", core.getOpenCount() == 1);
+    executor.execute(new NewSearcherRunnable(core));
+
+    // Since we called getCore() vs getCoreInc() and don't own a refCount, the container should decRef the core
+    // and close it when we call reload.
+    h.reload();
+
+    executor.shutdown();
+    executor.awaitTermination(1, TimeUnit.MINUTES);
+
+    // Check that all cores are closed and no searcher references are leaked.
+    assertTrue("SolrCore " + core + " is not closed", core.isClosed());
+    assertTrue(core.areAllSearcherReferencesEmpty());
+  }
+
+  private static class NewSearcherRunnable implements Runnable {
+    private final SolrCore core;
+
+    NewSearcherRunnable(SolrCore core) {
+      this.core = core;
+    }
+
+    @Override
+    public void run() {
+      while (!core.isClosed()) {
+        try {
+          RefCounted<SolrIndexSearcher> newSearcher = null;
+          try {
+            newSearcher = core.openNewSearcher(true, true);
+          } finally {
+            if (newSearcher != null) {
+              newSearcher.decref();
+            }
+          }
+        } catch (SolrException e) {
+          if (!core.isClosed()) {
+            throw e;
+          }
+        }
+      }
+    }
+  }
+
 }