You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by "Paul Ward (Jira)" <ji...@apache.org> on 2019/12/06 21:51:00 UTC

[jira] [Updated] (LUCENE-9084) circular synchronization wait (potential deadlock) in AnalyzingInfixSuggester

     [ https://issues.apache.org/jira/browse/LUCENE-9084?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Paul Ward updated LUCENE-9084:
------------------------------
    Attachment: LUCENE.patch

> circular synchronization wait (potential deadlock) in AnalyzingInfixSuggester
> -----------------------------------------------------------------------------
>
>                 Key: LUCENE-9084
>                 URL: https://issues.apache.org/jira/browse/LUCENE-9084
>             Project: Lucene - Core
>          Issue Type: Bug
>          Components: core/search
>            Reporter: Paul Ward
>            Priority: Major
>         Attachments: LUCENE.patch
>
>
> I created a pull request on github for this:
> https://github.com/apache/lucene-solr/pull/1064
> Detailed code (snippets and links) are in the sections after this overview (section **Detailed Code** and **This Patch's Code**).
> Method {{ensureOpen()}} is {{synchronized}} (acquires {{this}}) and its body contains a {{synchronized (searcherMgrLock)}} block (i.e., then acquires {{searcherMgrLock}}).
> Method {{ensureOpen()}} is called two times from public methods {{add()}} and {{update()}}.
> A thread calling public methods {{add()}} or {{update()}} will acquire locks in order:
> {code:java}
> this -> searcherMgrLock
> {code}
> Public method {{build()}} has a {{synchronized (searcherMgrLock)}} block in which it calls method {{add()}}.  Method {{add()}}, as described above, calls method {{ensureOpen()}}.
> Therefore, a thread calling public method {{build()}} will acquire locks in order:
> {code:java}
> searcherMgrLock -> this -> searcherMgrLock
> {code}
> 2 threads can acquire locks in different order which may cause a circular wait (deadlock).
> I do not know which threads call these methods, but there is a lot of synchronization in these methods and in this file, so I think these methods must be called concurrently.
> One thread can acquire:
> {{this -> searcherMgrLock}} (the first order above)
> and the other thread can acquire:
> {{searcherMgrLock -> this}} (the second order above).
> Note how the above 2 orders lead to a circular wait.
> h1. Detailed Code
> Method {{ensureOpen()}} is {{synchronized}} and its body contains a {{synchronized (searcherMgrLock)}}:
> {code:java}
>   private synchronized void ensureOpen() throws IOException { <<<<<<<<<<< see the synchronized keyword
>     if (writer == null) {
>       if (DirectoryReader.indexExists(dir)) {
>         // Already built; open it:
>         writer = new IndexWriter(dir, getIndexWriterConfig(getGramAnalyzer(), IndexWriterConfig.OpenMode.APPEND));
>       } else {
>         writer = new IndexWriter(dir, getIndexWriterConfig(getGramAnalyzer(), IndexWriterConfig.OpenMode.CREATE));
>       }
>       synchronized (searcherMgrLock) { <<<<<<<<<<<<<<<<<<<<<<<
> {code}
> https://github.com/apache/lucene-solr/blob/master/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java#L371-L379
> Method {{ensureOpen()}} is called two times from public methods {{add()}} and {{update()}}:
> {code:java}
>   public void add(BytesRef text, Set<BytesRef> contexts, long weight, BytesRef payload) throws IOException {
>     ensureOpen(); <<<<<<<<<<<<<<<<<<<<<<
> {code}
> https://github.com/apache/lucene-solr/blob/master/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java#L394-L395
> {code:java}
>   public void update(BytesRef text, Set<BytesRef> contexts, long weight, BytesRef payload) throws IOException {
>     ensureOpen(); <<<<<<<<<<<<<<<<<<<<
> {code}
> https://github.com/apache/lucene-solr/blob/master/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java#L406-L407
> Public method {{build()}} has a {{synchronized (searcherMgrLock)}} block in which it calls method {{add()}}:
> {code:java}
>   @Override
>   public void build(InputIterator iter) throws IOException {
>     
>     synchronized (searcherMgrLock) { <<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<<
>       if (searcherMgr != null) {
>         searcherMgr.close();
>         searcherMgr = null;
>       }
>       if (writer != null) {
>         writer.close();
>         writer = null;
>       }
>       boolean success = false;
>       try {
>         // First pass: build a temporary normal Lucene index,
>         // just indexing the suggestions as they iterate:
>         writer = new IndexWriter(dir,
>             getIndexWriterConfig(getGramAnalyzer(), IndexWriterConfig.OpenMode.CREATE));
>         //long t0 = System.nanoTime();
>         // TODO: use threads?
>         BytesRef text;
>         while ((text = iter.next()) != null) {
>           BytesRef payload;
>           if (iter.hasPayloads()) {
>             payload = iter.payload();
>           } else {
>             payload = null;
>           }
>           add(text, iter.contexts(), iter.weight(), payload);  <<<<<<<<<<<<<<<<<<<<<
> {code}
> https://github.com/apache/lucene-solr/blob/master/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java#L278-L310
> Method {{add()}} is the same one I linked above.
> h1. This Patch's Code
> Note that method {{ensureOpen()}} (inlined above) is the *only* place (method or synchronization block) that is {{synchronized}} on {{this}}.
> *All* the other synchronizations in this file are on {{searcherMgrLock}}.
> This CR removes the {{synchronized}} on {{this}} (again, being the only {{synchronized}} on {{this}}, we can do this change safely).
> And moves {{synchronized (searcherMgrLock)}} a few lines above,  to protect the entire code (that otherwise was protected by {{synchronized}} on {{this}}).
> The above breaks the lock cycle I described earlier.
> The fix looks big because it changes indentation.  
> But only one line is moved by a few lines up.
> I.e., from this:
> {code:java}
>   private synchronized void ensureOpen() throws IOException {
>     if (writer == null) {
>       if (DirectoryReader.indexExists(dir)) {
>         // Already built; open it:
>         writer = new IndexWriter(dir, getIndexWriterConfig(getGramAnalyzer(), IndexWriterConfig.OpenMode.APPEND));
>       } else {
>         writer = new IndexWriter(dir, getIndexWriterConfig(getGramAnalyzer(), IndexWriterConfig.OpenMode.CREATE));
>       }
>       synchronized (searcherMgrLock) { <<<<<<<<<<<<<<<<<<<<<< move from here
>         SearcherManager oldSearcherMgr = searcherMgr;
>         searcherMgr = new SearcherManager(writer, null);
>         if (oldSearcherMgr != null) {
>           oldSearcherMgr.close();
>         }
>       }
>     }
>   }
> {code}
> To this:
> {code:java}
>   private void ensureOpen() throws IOException { <<<<<<<<<<<<<<<<<< remove synchronized --- can lead to circular wait --- and legal to remove
>     synchronized (searcherMgrLock) { <<<<<<<<<<<<<<<<<<<<< move to here
>       if (writer == null) {
>         if (DirectoryReader.indexExists(dir)) {
>           // Already built; open it:
>           writer = new IndexWriter(dir, getIndexWriterConfig(getGramAnalyzer(), IndexWriterConfig.OpenMode.APPEND));
>         } else {
>           writer = new IndexWriter(dir, getIndexWriterConfig(getGramAnalyzer(), IndexWriterConfig.OpenMode.CREATE));
>         }
>         SearcherManager oldSearcherMgr = searcherMgr;
>         searcherMgr = new SearcherManager(writer, null);
>         if (oldSearcherMgr != null) {
>           oldSearcherMgr.close();
>         }
>       }
>     }
>   }
> {code}
> Here are all the places where {{synchronized (searcherMgrLock)}} appears in this file (and again, no other {{synchronized}} on other objects is done):
> - https://github.com/apache/lucene-solr/blob/master/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java#L281-L332
> - https://github.com/apache/lucene-solr/blob/master/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java#L379-L385
> - https://github.com/apache/lucene-solr/blob/master/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java#L654-L657
> - https://github.com/apache/lucene-solr/blob/master/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java#L870-L873
> - https://github.com/apache/lucene-solr/blob/master/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java#L898-L901
> - https://github.com/apache/lucene-solr/blob/master/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingInfixSuggester.java#L926-L929
> I.e., doing the synchronization like above is safe and consistent with the rest of the file.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

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