You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by Erick Erickson <er...@gmail.com> on 2013/02/25 14:25:07 UTC

Race condition in Solr, plea fo help and/or advice

OK, in working on SOLR-4196 I'm exercising opening/closing cores as never
before. I have a little stress program that does about the worst thing
possible, essentially opens and closes a core for every request. It has a
bunch of query and update threads running simultaneously that pick a random
core and do a query or update. I've got a bunch of code that, I think,
prevents any attempt to open _or_ close a core while it is being either
opened or closed by another thread (but I'm verifying).

It runs fine for a couple of hours, then hits a race condition. I was able
to get a stack trace (see below).

CloserThread.run(CoreContainer.java:1920) (second thread below) is, indeed,
new code. The stress-test program is updating cores (which may not be
loaded) like crazy and doing queries on other random cores. It's perfectly
possible to be updating a core that's in the process of being closed, I was
counting on the ref counting to make this OK... The cores are transient in
a limited cache, so they come and go. It looks like I'm trying to close a
core at the same time an update has come in, but I'm not sure whether this
is something that should be prevented from the new code or is an underlying
problem.

So a couple of questions:
1> SOLR-4196 has a whole series of improvements that even let us get here.
Running the stress test program against current trunk barfs before having
time to hit this condition, so the current state is an improvement. What do
you think about me checking 4196 in and opening a separate JIRA for this
issue?

2> Any suggestions on what direction to go next? If it's something easy, I
can just fold it into this patch.

3> Am I just going about things bass-ackwards? Not an unusual state of
affairs unfortunately.....

NOTE: The current patch for SOLR-4196 isn't the one running with this code,
there are a couple more things I want change. Mostly I'm asking if someone
familiar with the code where the race is encountered has a quick fix....

Thanks,
Erick


Found one Java-level deadlock:
=============================
"commitScheduler-122579-thread-1":
  waiting to lock monitor 7f87c3076d38 (object 78b379a28, a
org.apache.solr.update.DefaultSolrCoreState),
  which is held by "Thread-15"
"Thread-15":
  waiting for ownable synchronizer 765e84638, (a
java.util.concurrent.locks.ReentrantLock$NonfairSync),
  which is held by "commitScheduler-122579-thread-1"

Java stack information for the threads listed above:
===================================================
"commitScheduler-122579-thread-1":
at
org.apache.solr.update.DefaultSolrCoreState.getIndexWriter(DefaultSolrCoreState.java:82)
- waiting to lock <78b379a28> (a
org.apache.solr.update.DefaultSolrCoreState)
at org.apache.solr.core.SolrCore.openNewSearcher(SolrCore.java:1354)
at
org.apache.solr.update.DirectUpdateHandler2.commit(DirectUpdateHandler2.java:573)
- locked <76aa46f58> (a java.lang.Object)
at org.apache.solr.update.CommitTracker.run(CommitTracker.java:216)
at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:439)
at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
at java.util.concurrent.FutureTask.run(FutureTask.java:138)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:98)
at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:206)
at
java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
at java.lang.Thread.run(Thread.java:680)
 "Thread-15":
at sun.misc.Unsafe.park(Native Method)
- parking to wait for  <765e84638> (a
java.util.concurrent.locks.ReentrantLock$NonfairSync)
at java.util.concurrent.locks.LockSupport.park(LockSupport.java:156)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:811)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireQueued(AbstractQueuedSynchronizer.java:842)
at
java.util.concurrent.locks.AbstractQueuedSynchronizer.acquire(AbstractQueuedSynchronizer.java:1178)
at
java.util.concurrent.locks.ReentrantLock$NonfairSync.lock(ReentrantLock.java:186)
at java.util.concurrent.locks.ReentrantLock.lock(ReentrantLock.java:262)
at
org.apache.solr.update.DirectUpdateHandler2.closeWriter(DirectUpdateHandler2.java:680)
at
org.apache.solr.update.DefaultSolrCoreState.closeIndexWriter(DefaultSolrCoreState.java:68)
- locked <78b379a28> (a org.apache.solr.update.DefaultSolrCoreState)
at
org.apache.solr.update.DefaultSolrCoreState.close(DefaultSolrCoreState.java:289)
- locked <78b379a28> (a org.apache.solr.update.DefaultSolrCoreState)
at
org.apache.solr.update.SolrCoreState.decrefSolrCoreState(SolrCoreState.java:68)
- locked <78b379a28> (a org.apache.solr.update.DefaultSolrCoreState)
at org.apache.solr.core.SolrCore.close(SolrCore.java:975)
at org.apache.solr.core.CloserThread.run(CoreContainer.java:1920)

Found 1 deadlock.

Re: Race condition in Solr, plea fo help and/or advice

Posted by Mark Miller <ma...@gmail.com>.
FYI, I filed https://issues.apache.org/jira/browse/SOLR-4505

- Mark

On Feb 26, 2013, at 12:16 AM, Mark Miller <ma...@gmail.com> wrote:

> You are finding great stuff with this stress test driver! We really need this in the tests :)
> 
> On Feb 25, 2013, at 8:25 AM, Erick Erickson <er...@gmail.com> wrote:
> 
>> What do you think about me checking 4196 in and opening a separate JIRA for this issue?
> 
> I think that is fine - I don't think the deadlock is related to 4196.
> 
> I think the following patch will address it:
> 
> Index: solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
> ===================================================================
> --- solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java	(revision 1450011)
> +++ solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java	(working copy)
> @@ -348,10 +348,13 @@
> 
>       // currently for testing purposes.  Do a delete of complete index w/o worrying about versions, don't log, clean up most state in update log, etc
>       if (delAll && cmd.getVersion() == -Long.MAX_VALUE) {
> -        synchronized (solrCoreState.getUpdateLock()) {
> +        solrCoreState.getUpdateLock().lock();
> +        try {
>           deleteAll();
>           ulog.deleteAll();
>           return;
> +        } finally {
> +          solrCoreState.getUpdateLock().unlock();
>         }
>       }
> 
> @@ -361,7 +364,8 @@
>       // a realtime view of the index.  When a new searcher is opened after a DBQ, that
>       // flag can be cleared.  If those thing happen concurrently, it's not thread safe.
>       //
> -      synchronized (solrCoreState.getUpdateLock()) {
> +      solrCoreState.getUpdateLock().lock();
> +      try {
>         if (delAll) {
>           deleteAll();
>         } else {
> @@ -374,6 +378,8 @@
>         }
> 
>         if (ulog != null) ulog.deleteByQuery(cmd);
> +      } finally {
> +        solrCoreState.getUpdateLock().unlock();
>       }
> 
>       madeIt = true;
> @@ -397,7 +403,8 @@
>     Term idTerm = new Term(idField.getName(), cmd.getIndexedId());
> 
>     // see comment in deleteByQuery
> -    synchronized (solrCoreState.getUpdateLock()) {
> +    solrCoreState.getUpdateLock().lock();
> +    try {
>       RefCounted<IndexWriter> iw = solrCoreState.getIndexWriter(core);
>       try {
>         IndexWriter writer = iw.get();
> @@ -412,6 +419,8 @@
>       }
> 
>       if (ulog != null) ulog.add(cmd, true);
> +    } finally {
> +      solrCoreState.getUpdateLock().unlock();
>     }
> 
>   }
> @@ -527,10 +536,13 @@
>         }
> 
>         if (!cmd.softCommit) {
> -          synchronized (solrCoreState.getUpdateLock()) { // sync is currently needed to prevent preCommit
> -                                // from being called between preSoft and
> -                                // postSoft... see postSoft comments.
> +          solrCoreState.getUpdateLock().lock(); // sync is currently needed to prevent preCommit
> +                                                // from being called between preSoft and
> +                                                // postSoft... see postSoft comments.
> +          try {
>             if (ulog != null) ulog.preCommit(cmd);
> +          } finally {
> +            solrCoreState.getUpdateLock().unlock();
>           }
> 
>           // SolrCore.verbose("writer.commit() start writer=",writer);
> @@ -557,14 +569,18 @@
> 
>       if (cmd.softCommit) {
>         // ulog.preSoftCommit();
> -        synchronized (solrCoreState.getUpdateLock()) {
> +        solrCoreState.getUpdateLock().lock();
> +        try {
>           if (ulog != null) ulog.preSoftCommit(cmd);
>           core.getSearcher(true, false, waitSearcher, true);
>           if (ulog != null) ulog.postSoftCommit(cmd);
> +        } finally {
> +          solrCoreState.getUpdateLock().unlock();
>         }
>         // ulog.postSoftCommit();
>       } else {
> -        synchronized (solrCoreState.getUpdateLock()) {
> +        solrCoreState.getUpdateLock().lock();
> +        try {
>           if (ulog != null) ulog.preSoftCommit(cmd);
>           if (cmd.openSearcher) {
>             core.getSearcher(true, false, waitSearcher);
> @@ -574,6 +590,8 @@
>             searchHolder.decref();
>           }
>           if (ulog != null) ulog.postSoftCommit(cmd);
> +        } finally {
> +          solrCoreState.getUpdateLock().unlock();
>         }
>         if (ulog != null) ulog.postCommit(cmd); // postCommit currently means new searcher has
>                               // also been opened
> @@ -713,8 +731,11 @@
>           // TODO: keep other commit callbacks from being called?
>          //  this.commit(cmd);        // too many test failures using this method... is it because of callbacks?
> 
> -          synchronized (solrCoreState.getUpdateLock()) {
> +          solrCoreState.getUpdateLock().lock();
> +          try {
>             ulog.preCommit(cmd);
> +          } finally {
> +            solrCoreState.getUpdateLock().unlock();
>           }
> 
>           // todo: refactor this shared code (or figure out why a real CommitUpdateCommand can't be used)
> @@ -723,8 +744,11 @@
>           writer.setCommitData(commitData);
>           writer.commit();
> 
> -          synchronized (solrCoreState.getUpdateLock()) {
> +          solrCoreState.getUpdateLock().lock();
> +          try {
>             ulog.postCommit(cmd);
> +          } finally {
> +            solrCoreState.getUpdateLock().unlock();
>           }
>         }
>       } catch (Throwable th) {
> Index: solr/core/src/java/org/apache/solr/update/SolrCoreState.java
> ===================================================================
> --- solr/core/src/java/org/apache/solr/update/SolrCoreState.java	(revision 1450012)
> +++ solr/core/src/java/org/apache/solr/update/SolrCoreState.java	(working copy)
> @@ -37,12 +37,6 @@
> public abstract class SolrCoreState {
>   public static Logger log = LoggerFactory.getLogger(SolrCoreState.class);
> 
> -  private final Object deleteLock = new Object();
> -  
> -  public Object getUpdateLock() {
> -    return deleteLock;
> -  }
> -  
>   private int solrCoreStateRefCnt = 1;
> 
>   public synchronized int getSolrCoreStateRefCnt() {
> @@ -76,6 +70,7 @@
>   }
> 
>   public abstract Lock getCommitLock();
> +  public abstract Lock getUpdateLock();
> 
>   /**
>    * Force the creation of a new IndexWriter using the settings from the given
> Index: solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
> ===================================================================
> --- solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java	(revision 1450012)
> +++ solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java	(working copy)
> @@ -56,6 +56,7 @@
>   private boolean writerFree = true;
> 
>   protected final ReentrantLock commitLock = new ReentrantLock();
> +  protected final ReentrantLock updateLock = new ReentrantLock();
> 
>   public DefaultSolrCoreState(DirectoryFactory directoryFactory) {
>     this.directoryFactory = directoryFactory;
> @@ -142,11 +143,16 @@
>       log.info("Waiting until IndexWriter is unused... core=" + coreName);
> 
>       boolean yieldedCommitLock = false;
> +      boolean yieldedUpdateLock = false;
>       try {
>         if (commitLock.isHeldByCurrentThread()) {
>           yieldedCommitLock = true;
>           commitLock.unlock();
>         }
> +        if (updateLock.isHeldByCurrentThread()) {
> +          yieldedUpdateLock = true;
> +          updateLock.unlock();
> +        }
> 
>         while (!writerFree) {
>           try {
> @@ -161,6 +167,9 @@
>         if (yieldedCommitLock) {
>           commitLock.lock();
>         }
> +        if (yieldedUpdateLock) {
> +          updateLock.lock();
> +        }
>       }
> 
>       try {
> @@ -295,4 +304,9 @@
>     return commitLock;
>   }
> 
> +  @Override
> +  public Lock getUpdateLock() {
> +    return updateLock;
> +  }
> +  
> }
> 


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


Re: Race condition in Solr, plea fo help and/or advice

Posted by Mark Miller <ma...@gmail.com>.
You are finding great stuff with this stress test driver! We really need this in the tests :)

On Feb 25, 2013, at 8:25 AM, Erick Erickson <er...@gmail.com> wrote:

> What do you think about me checking 4196 in and opening a separate JIRA for this issue?

I think that is fine - I don't think the deadlock is related to 4196.

I think the following patch will address it:

Index: solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
===================================================================
--- solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java	(revision 1450011)
+++ solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java	(working copy)
@@ -348,10 +348,13 @@
 
       // currently for testing purposes.  Do a delete of complete index w/o worrying about versions, don't log, clean up most state in update log, etc
       if (delAll && cmd.getVersion() == -Long.MAX_VALUE) {
-        synchronized (solrCoreState.getUpdateLock()) {
+        solrCoreState.getUpdateLock().lock();
+        try {
           deleteAll();
           ulog.deleteAll();
           return;
+        } finally {
+          solrCoreState.getUpdateLock().unlock();
         }
       }
 
@@ -361,7 +364,8 @@
       // a realtime view of the index.  When a new searcher is opened after a DBQ, that
       // flag can be cleared.  If those thing happen concurrently, it's not thread safe.
       //
-      synchronized (solrCoreState.getUpdateLock()) {
+      solrCoreState.getUpdateLock().lock();
+      try {
         if (delAll) {
           deleteAll();
         } else {
@@ -374,6 +378,8 @@
         }
 
         if (ulog != null) ulog.deleteByQuery(cmd);
+      } finally {
+        solrCoreState.getUpdateLock().unlock();
       }
 
       madeIt = true;
@@ -397,7 +403,8 @@
     Term idTerm = new Term(idField.getName(), cmd.getIndexedId());
     
     // see comment in deleteByQuery
-    synchronized (solrCoreState.getUpdateLock()) {
+    solrCoreState.getUpdateLock().lock();
+    try {
       RefCounted<IndexWriter> iw = solrCoreState.getIndexWriter(core);
       try {
         IndexWriter writer = iw.get();
@@ -412,6 +419,8 @@
       }
       
       if (ulog != null) ulog.add(cmd, true);
+    } finally {
+      solrCoreState.getUpdateLock().unlock();
     }
     
   }
@@ -527,10 +536,13 @@
         }
         
         if (!cmd.softCommit) {
-          synchronized (solrCoreState.getUpdateLock()) { // sync is currently needed to prevent preCommit
-                                // from being called between preSoft and
-                                // postSoft... see postSoft comments.
+          solrCoreState.getUpdateLock().lock(); // sync is currently needed to prevent preCommit
+                                                // from being called between preSoft and
+                                                // postSoft... see postSoft comments.
+          try {
             if (ulog != null) ulog.preCommit(cmd);
+          } finally {
+            solrCoreState.getUpdateLock().unlock();
           }
           
           // SolrCore.verbose("writer.commit() start writer=",writer);
@@ -557,14 +569,18 @@
 
       if (cmd.softCommit) {
         // ulog.preSoftCommit();
-        synchronized (solrCoreState.getUpdateLock()) {
+        solrCoreState.getUpdateLock().lock();
+        try {
           if (ulog != null) ulog.preSoftCommit(cmd);
           core.getSearcher(true, false, waitSearcher, true);
           if (ulog != null) ulog.postSoftCommit(cmd);
+        } finally {
+          solrCoreState.getUpdateLock().unlock();
         }
         // ulog.postSoftCommit();
       } else {
-        synchronized (solrCoreState.getUpdateLock()) {
+        solrCoreState.getUpdateLock().lock();
+        try {
           if (ulog != null) ulog.preSoftCommit(cmd);
           if (cmd.openSearcher) {
             core.getSearcher(true, false, waitSearcher);
@@ -574,6 +590,8 @@
             searchHolder.decref();
           }
           if (ulog != null) ulog.postSoftCommit(cmd);
+        } finally {
+          solrCoreState.getUpdateLock().unlock();
         }
         if (ulog != null) ulog.postCommit(cmd); // postCommit currently means new searcher has
                               // also been opened
@@ -713,8 +731,11 @@
           // TODO: keep other commit callbacks from being called?
          //  this.commit(cmd);        // too many test failures using this method... is it because of callbacks?
 
-          synchronized (solrCoreState.getUpdateLock()) {
+          solrCoreState.getUpdateLock().lock();
+          try {
             ulog.preCommit(cmd);
+          } finally {
+            solrCoreState.getUpdateLock().unlock();
           }
 
           // todo: refactor this shared code (or figure out why a real CommitUpdateCommand can't be used)
@@ -723,8 +744,11 @@
           writer.setCommitData(commitData);
           writer.commit();
 
-          synchronized (solrCoreState.getUpdateLock()) {
+          solrCoreState.getUpdateLock().lock();
+          try {
             ulog.postCommit(cmd);
+          } finally {
+            solrCoreState.getUpdateLock().unlock();
           }
         }
       } catch (Throwable th) {
Index: solr/core/src/java/org/apache/solr/update/SolrCoreState.java
===================================================================
--- solr/core/src/java/org/apache/solr/update/SolrCoreState.java	(revision 1450012)
+++ solr/core/src/java/org/apache/solr/update/SolrCoreState.java	(working copy)
@@ -37,12 +37,6 @@
 public abstract class SolrCoreState {
   public static Logger log = LoggerFactory.getLogger(SolrCoreState.class);
   
-  private final Object deleteLock = new Object();
-  
-  public Object getUpdateLock() {
-    return deleteLock;
-  }
-  
   private int solrCoreStateRefCnt = 1;
   
   public synchronized int getSolrCoreStateRefCnt() {
@@ -76,6 +70,7 @@
   }
   
   public abstract Lock getCommitLock();
+  public abstract Lock getUpdateLock();
   
   /**
    * Force the creation of a new IndexWriter using the settings from the given
Index: solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
===================================================================
--- solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java	(revision 1450012)
+++ solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java	(working copy)
@@ -56,6 +56,7 @@
   private boolean writerFree = true;
   
   protected final ReentrantLock commitLock = new ReentrantLock();
+  protected final ReentrantLock updateLock = new ReentrantLock();
 
   public DefaultSolrCoreState(DirectoryFactory directoryFactory) {
     this.directoryFactory = directoryFactory;
@@ -142,11 +143,16 @@
       log.info("Waiting until IndexWriter is unused... core=" + coreName);
       
       boolean yieldedCommitLock = false;
+      boolean yieldedUpdateLock = false;
       try {
         if (commitLock.isHeldByCurrentThread()) {
           yieldedCommitLock = true;
           commitLock.unlock();
         }
+        if (updateLock.isHeldByCurrentThread()) {
+          yieldedUpdateLock = true;
+          updateLock.unlock();
+        }
         
         while (!writerFree) {
           try {
@@ -161,6 +167,9 @@
         if (yieldedCommitLock) {
           commitLock.lock();
         }
+        if (yieldedUpdateLock) {
+          updateLock.lock();
+        }
       }
 
       try {
@@ -295,4 +304,9 @@
     return commitLock;
   }
   
+  @Override
+  public Lock getUpdateLock() {
+    return updateLock;
+  }
+  
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org