mirror of https://github.com/apache/lucene.git
SOLR-509: roll back prev changes that caused firstSearcher to never be closed, use latch to prevent events from executing before inform() is called
git-svn-id: https://svn.apache.org/repos/asf/lucene/solr/trunk@653091 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
72869fb61d
commit
f3ed5f8424
|
@ -17,45 +17,24 @@
|
|||
|
||||
package org.apache.solr.core;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Collections;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.Executors;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.logging.Logger;
|
||||
|
||||
import javax.xml.parsers.ParserConfigurationException;
|
||||
import javax.xml.xpath.XPathConstants;
|
||||
|
||||
import org.apache.lucene.index.IndexReader;
|
||||
import org.apache.lucene.search.BooleanQuery;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.FSDirectory;
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.params.CommonParams;
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.apache.solr.common.params.CommonParams.EchoParamStyle;
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.apache.solr.common.util.DOMUtil;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||
import org.apache.solr.handler.component.DebugComponent;
|
||||
import org.apache.solr.handler.component.FacetComponent;
|
||||
import org.apache.solr.handler.component.HighlightComponent;
|
||||
import org.apache.solr.handler.component.MoreLikeThisComponent;
|
||||
import org.apache.solr.handler.component.QueryComponent;
|
||||
import org.apache.solr.handler.component.SearchComponent;
|
||||
import org.apache.solr.handler.component.*;
|
||||
import org.apache.solr.highlight.DefaultSolrHighlighter;
|
||||
import org.apache.solr.highlight.SolrHighlighter;
|
||||
import org.apache.solr.request.*;
|
||||
import org.apache.solr.schema.IndexSchema;
|
||||
import org.apache.solr.search.SolrIndexSearcher;
|
||||
import org.apache.solr.search.QParserPlugin;
|
||||
import org.apache.solr.search.SolrIndexSearcher;
|
||||
import org.apache.solr.search.ValueSourceParser;
|
||||
import org.apache.solr.update.DirectUpdateHandler;
|
||||
import org.apache.solr.update.SolrIndexWriter;
|
||||
|
@ -69,6 +48,14 @@ import org.w3c.dom.Node;
|
|||
import org.w3c.dom.NodeList;
|
||||
import org.xml.sax.SAXException;
|
||||
|
||||
import javax.xml.parsers.ParserConfigurationException;
|
||||
import javax.xml.xpath.XPathConstants;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.*;
|
||||
import java.util.concurrent.*;
|
||||
import java.util.logging.Logger;
|
||||
|
||||
|
||||
/**
|
||||
* @version $Id$
|
||||
|
@ -380,48 +367,44 @@ public final class SolrCore {
|
|||
solrConfig.get("highlighting/@class", DefaultSolrHighlighter.class.getName())
|
||||
);
|
||||
highlighter.initalize( solrConfig );
|
||||
|
||||
|
||||
final CountDownLatch latch = new CountDownLatch(1);
|
||||
|
||||
try {
|
||||
// Open the searcher *before* the handler so we don't end up opening
|
||||
// cause the executor to stall so firstSearcher events won't fire
|
||||
// until after inform() has been called for all components.
|
||||
// searchExecutor must be single-threaded for this to work
|
||||
searcherExecutor.submit(new Callable() {
|
||||
public Object call() throws Exception {
|
||||
latch.await();
|
||||
return null;
|
||||
}
|
||||
});
|
||||
|
||||
// Open the searcher *before* the update handler so we don't end up opening
|
||||
// one in the middle.
|
||||
// With lockless commits in Lucene now, this probably shouldn't be an issue anymore
|
||||
getSearcher(false,false,null);
|
||||
|
||||
updateHandler = createUpdateHandler(
|
||||
solrConfig.get("updateHandler/@class", DirectUpdateHandler.class.getName())
|
||||
);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
|
||||
// Finally tell anyone who wants to know
|
||||
loader.inform( loader );
|
||||
loader.inform( this );
|
||||
|
||||
// execute firstSearcher event
|
||||
//TODO: It may not always be the case that this is the only time the first searcher event needs to fire.
|
||||
doFirstSearcherEvent(getSearcher().get());
|
||||
}
|
||||
|
||||
// Finally tell anyone who wants to know
|
||||
loader.inform( loader );
|
||||
loader.inform( this );
|
||||
|
||||
} catch (IOException e) {
|
||||
throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
|
||||
} finally {
|
||||
// allow firstSearcher events to fire
|
||||
latch.countDown();
|
||||
}
|
||||
} // end synchronized
|
||||
}
|
||||
|
||||
private void doFirstSearcherEvent(final SolrIndexSearcher firstSearcher){
|
||||
if (firstSearcherListeners.size() > 0) {
|
||||
searcherExecutor.submit(
|
||||
new Callable() {
|
||||
public Object call() throws Exception {
|
||||
try {
|
||||
for (SolrEventListener listener : firstSearcherListeners) {
|
||||
listener.newSearcher(firstSearcher,null);
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
SolrException.logOnce(log,null,e);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Load the request processors configured in solrconfig.xml
|
||||
|
@ -811,6 +794,23 @@ public final class SolrCore {
|
|||
}
|
||||
);
|
||||
}
|
||||
|
||||
if (currSearcher==null && firstSearcherListeners.size() > 0) {
|
||||
future = searcherExecutor.submit(
|
||||
new Callable() {
|
||||
public Object call() throws Exception {
|
||||
try {
|
||||
for (SolrEventListener listener : firstSearcherListeners) {
|
||||
listener.newSearcher(newSearcher,null);
|
||||
}
|
||||
} catch (Throwable e) {
|
||||
SolrException.logOnce(log,null,e);
|
||||
}
|
||||
return null;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
if (currSearcher!=null && newSearcherListeners.size() > 0) {
|
||||
future = searcherExecutor.submit(
|
||||
|
|
Loading…
Reference in New Issue